Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Arrow: Fix indexing in Parquet dictionary encoded values readers #11247

Open
wants to merge 5 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -59,14 +59,10 @@ public void nextBatch(
}
int numValues = Math.min(left, currentCount);
for (int i = 0; i < numValues; i++) {
int index = idx * typeWidth;
if (typeWidth == -1) {
index = idx;
}
Comment on lines -63 to -65
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This happens in the case of var width binary, so the call to VarWidthBinaryDictEncodedReader::nextVal works correctly in that case.
Of course, it still works correctly with the change.

if (Mode.RLE.equals(mode)) {
nextVal(vector, dict, index, currentValue, typeWidth);
nextVal(vector, dict, idx, currentValue, typeWidth);
} else if (Mode.PACKED.equals(mode)) {
nextVal(vector, dict, index, packedValuesBuffer[packedValuesBufferIdx++], typeWidth);
nextVal(vector, dict, idx, packedValuesBuffer[packedValuesBufferIdx++], typeWidth);
}
nullabilityHolder.setNotNull(idx);
if (setArrowValidityVector) {
Expand Down Expand Up @@ -95,15 +91,15 @@ class LongDictEncodedReader extends BaseDictEncodedReader {
@Override
protected void nextVal(
FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) {
vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentVal));
vector.getDataBuffer().setLong((long) idx * typeWidth, dict.decodeToLong(currentVal));
}
}

class TimestampMillisDictEncodedReader extends BaseDictEncodedReader {
@Override
protected void nextVal(
FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) {
vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentVal) * 1000);
vector.getDataBuffer().setLong((long) idx * typeWidth, dict.decodeToLong(currentVal) * 1000);
}
}

Expand All @@ -114,31 +110,31 @@ protected void nextVal(
ByteBuffer buffer =
dict.decodeToBinary(currentVal).toByteBuffer().order(ByteOrder.LITTLE_ENDIAN);
long timestampInt96 = ParquetUtil.extractTimestampInt96(buffer);
vector.getDataBuffer().setLong(idx, timestampInt96);
vector.getDataBuffer().setLong((long) idx * typeWidth, timestampInt96);
}
}

class IntegerDictEncodedReader extends BaseDictEncodedReader {
@Override
protected void nextVal(
FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) {
vector.getDataBuffer().setInt(idx, dict.decodeToInt(currentVal));
vector.getDataBuffer().setInt((long) idx * typeWidth, dict.decodeToInt(currentVal));
}
}

class FloatDictEncodedReader extends BaseDictEncodedReader {
@Override
protected void nextVal(
FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) {
vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentVal));
vector.getDataBuffer().setFloat((long) idx * typeWidth, dict.decodeToFloat(currentVal));
}
}

class DoubleDictEncodedReader extends BaseDictEncodedReader {
@Override
protected void nextVal(
FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) {
vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentVal));
vector.getDataBuffer().setDouble((long) idx * typeWidth, dict.decodeToDouble(currentVal));
}
}

Expand All @@ -147,7 +143,7 @@ class FixedWidthBinaryDictEncodedReader extends BaseDictEncodedReader {
protected void nextVal(
FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) {
ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer();
vector.getDataBuffer().setBytes(idx, buffer);
vector.getDataBuffer().setBytes((long) idx * typeWidth, buffer);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,21 @@ public static void assertEqualsBatch(
}
}

public static void assertEqualsBatchWithRows(
Types.StructType struct, Iterator<Row> expected, ColumnarBatch batch) {
for (int rowId = 0; rowId < batch.numRows(); rowId++) {
List<Types.NestedField> fields = struct.fields();
InternalRow row = batch.getRow(rowId);
Row expectedRow = expected.next();
for (int i = 0; i < fields.size(); i += 1) {
Type fieldType = fields.get(i).type();
Object expectedValue = expectedRow.get(i);
Object actualValue = row.isNullAt(i) ? null : row.get(i, convert(fieldType));
assertEqualsUnsafe(fieldType, expectedValue, actualValue);
}
}
}

private static void assertEqualsSafe(Types.ListType list, Collection<?> expected, List actual) {
Type elementType = list.elementType();
List<?> expectedElements = Lists.newArrayList(expected);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,21 @@
*/
package org.apache.iceberg.spark.data.parquet.vectorized;

import static org.apache.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.PARQUET_PAGE_ROW_LIMIT;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT;
import static org.assertj.core.api.Assertions.assertThat;

import java.io.File;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Iterator;
import java.util.List;
import org.apache.avro.generic.GenericData;
import org.apache.iceberg.Files;
import org.apache.iceberg.Schema;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.parquet.Parquet;
import org.apache.iceberg.relocated.com.google.common.base.Function;
Expand All @@ -33,11 +41,35 @@
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.apache.iceberg.spark.data.RandomData;
import org.apache.iceberg.spark.data.TestHelpers;
import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders;
import org.apache.iceberg.types.Types;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.vectorized.ColumnarBatch;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;

public class TestParquetDictionaryEncodedVectorizedReads extends TestParquetVectorizedReads {

protected static SparkSession spark = null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we test this without having to start up spark? Also it seems like IntBackedDecimalDictEncodedReader/ LongBackedDecimalDictEncodedReader / VarWidthBinaryDictEncodedReader / FixedLengthDecimalDictEncodedReader have been affected, so maybe there should be tests for these too?

Copy link
Contributor Author

@wypoon wypoon Oct 7, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Using the Spark DataFrameReader to read the Parquet file is the easiest way for the test verification.

IIUC, if a decimal can be represented by 4 bytes, we use IntBackedDecimalDictEncodedReader and if it cannot be represented by 4 bytes but can be by 8 bytes, we use LongBackedDecimalDictEncodedReader, but can you please tell me what types use FixedLengthDecimalDictEncodedReader?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, afaics, VectorizedParquetDefinitionLevelReader.FixedLengthDecimalReader is never used, and thus neither is VectorizedDictionaryEncodedParquetValuesReader.FixedLengthDecimalDictEncodedReader.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, even VectorizedParquetDefinitionLevelReader.IntBackedDecimalReader and VectorizedParquetDefinitionLevelReader.LongBackedDecimalReader are never used, afait.
In VectorizedParquetDefinitionLevelReader, the methods fixedLengthDecimalReader(), intBackedDecimalReader() and longBackedDecimalReader() are never called.

I see now that VectorizedPageIterator.IntBackedDecimalPageReader, VectorizedPageIterator.LongBackedDecimalPageReader, VectorizedPageIterator::intBackedDecimalPageReader(), and VectorizedPageIterator::longBackedDecimalPageReader() were deprecated in #3249 and subsequently all removed in Iceberg 1.4.0. So we have some dead code now.


@BeforeAll
public static void startSpark() {
spark = SparkSession.builder().master("local[2]").getOrCreate();
}

@AfterAll
public static void stopSpark() {
if (spark != null) {
spark.stop();
spark = null;
}
}

@Override
Iterable<GenericData.Record> generateData(
Schema schema,
Expand Down Expand Up @@ -93,4 +125,64 @@ public void testMixedDictionaryNonDictionaryReads() throws IOException {
true,
BATCH_SIZE);
}

@Test
public void testBinaryNotAllPagesDictionaryEncoded() throws IOException {
Schema schema = new Schema(Types.NestedField.required(1, "bytes", Types.BinaryType.get()));
File parquetFile = File.createTempFile("junit", null, temp.toFile());
assertThat(parquetFile.delete()).as("Delete should succeed").isTrue();

Iterable<GenericData.Record> records = RandomData.generateFallbackData(schema, 500, 0L, 100);
try (FileAppender<GenericData.Record> writer =
Parquet.write(Files.localOutput(parquetFile))
.schema(schema)
.set(PARQUET_DICT_SIZE_BYTES, "4096")
.set(PARQUET_PAGE_ROW_LIMIT, "100")
.build()) {
writer.addAll(records);
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change

// After the above, parquetFile contains one column chunk of binary data in five pages,
// the first two RLE dictionary encoded, and the remaining three plain encoded.
assertRecordsMatch(schema, 500, records, parquetFile, true, BATCH_SIZE);
}

/**
* decimal_dict_and_plain_encoding.parquet contains one column chunk of decimal(38, 0) data in two
* pages, one RLE dictionary encoded and one plain encoded, each with 200 rows.
*/
@Test
public void testDecimalNotAllPagesDictionaryEncoded() throws Exception {
Schema schema = new Schema(Types.NestedField.required(1, "id", Types.DecimalType.of(38, 0)));
Path path =
Paths.get(
getClass()
.getClassLoader()
.getResource("decimal_dict_and_plain_encoding.parquet")
.toURI());
Comment on lines +157 to +162
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there any way we can generate this parquet file in the test locally via existing writers instead of uploading this resource?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just saw this thread #11247 (comment) let me go through it

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@amogh-jahagirdar I have already explained this at length and convinced @nastra.


Dataset<Row> df = spark.read().parquet(path.toString());
List<Row> expected = df.collectAsList();
long expectedSize = df.count();

Parquet.ReadBuilder readBuilder =
Parquet.read(Files.localInput(path.toFile()))
.project(schema)
.createBatchedReaderFunc(
type ->
VectorizedSparkParquetReaders.buildReader(
schema, type, ImmutableMap.of(), null));

try (CloseableIterable<ColumnarBatch> batchReader = readBuilder.build()) {
Iterator<Row> expectedIter = expected.iterator();
Iterator<ColumnarBatch> batches = batchReader.iterator();
int numRowsRead = 0;
while (batches.hasNext()) {
ColumnarBatch batch = batches.next();
numRowsRead += batch.numRows();
TestHelpers.assertEqualsBatchWithRows(schema.asStruct(), expectedIter, batch);
}
assertThat(numRowsRead).isEqualTo(expectedSize);
}
}
}
Binary file not shown.