Thanks to visit codestin.com
Credit goes to github.com

Skip to content
3 changes: 3 additions & 0 deletions core/src/main/java/org/apache/iceberg/ManifestReader.java
Original file line number Diff line number Diff line change
Expand Up @@ -278,6 +278,9 @@ private CloseableIterable<ManifestEntry<F>> open(Schema projection) {
if (projection.findField(DataFile.RECORD_COUNT.fieldId()) == null) {
fields.add(DataFile.RECORD_COUNT);
}
if (projection.findField(DataFile.FIRST_ROW_ID.fieldId()) == null) {
fields.add(DataFile.FIRST_ROW_ID);
}
fields.add(MetadataColumns.ROW_POSITION);

CloseableIterable<ManifestEntry<F>> reader =
Expand Down
11 changes: 11 additions & 0 deletions core/src/main/java/org/apache/iceberg/util/PartitionUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,17 @@ private PartitionUtil() {}
// use java.util.HashMap because partition data may contain null values
Map<Integer, Object> idToConstant = Maps.newHashMap();

// add first_row_id as _row_id
if (task.file().firstRowId() != null) {
idToConstant.put(
MetadataColumns.ROW_ID.fieldId(),
convertConstant.apply(Types.LongType.get(), task.file().firstRowId()));
}

idToConstant.put(
MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(),
convertConstant.apply(Types.LongType.get(), task.file().fileSequenceNumber()));

// add _file
idToConstant.put(
MetadataColumns.FILE_PATH.fieldId(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -685,6 +685,9 @@ private static void checkDataFileAssignment(
try (ManifestReader<DataFile> reader =
ManifestFiles.read(manifest, table.io(), table.specs())) {

// test that the first_row_id column is always scanned, even if not requested
reader.select(BaseScan.SCAN_COLUMNS);

for (DataFile file : reader) {
assertThat(file.content()).isEqualTo(FileContent.DATA);
if (index < firstRowIds.length) {
Expand Down
48 changes: 48 additions & 0 deletions core/src/test/java/org/apache/iceberg/data/DataTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,11 @@
import java.nio.ByteBuffer;
import java.nio.file.Path;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Stream;
import org.apache.iceberg.MetadataColumns;
import org.apache.iceberg.Schema;
import org.apache.iceberg.expressions.Literal;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
Expand All @@ -53,6 +55,14 @@

public abstract class DataTest {

private static final long FIRST_ROW_ID = 2_000L;
protected static final Map<Integer, Object> ID_TO_CONSTANT =
Map.of(
MetadataColumns.ROW_ID.fieldId(),
FIRST_ROW_ID,
MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(),
34L);

protected abstract void writeAndValidate(Schema schema) throws IOException;

protected void writeAndValidate(Schema schema, List<Record> data) throws IOException {
Expand Down Expand Up @@ -139,6 +149,10 @@ protected boolean supportsGeospatial() {
return false;
}

protected boolean supportsRowLineage() {
return false;
}

@ParameterizedTest
@FieldSource("SIMPLE_TYPES")
public void testTypeSchema(Type type) throws IOException {
Expand Down Expand Up @@ -599,4 +613,38 @@ public void testWriteNullValueForRequiredType() throws Exception {
() -> writeAndValidate(schema, ImmutableList.of(genericRecord)));
}
}

@Test
public void testRowLineage() throws Exception {
Assumptions.assumeThat(supportsRowLineage())
.as("Row Lineage support is not implemented")
.isTrue();

Schema schema =
new Schema(
required(1, "id", LongType.get()),
required(2, "data", Types.StringType.get()),
MetadataColumns.ROW_ID,
MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER);

GenericRecord record = GenericRecord.create(schema);

writeAndValidate(
schema,
List.of(
record.copy(Map.of("id", 1L, "data", "a")),
record.copy(Map.of("id", 2L, "data", "b")),
record.copy(
Map.of(
"id",
3L,
"data",
"c",
"_row_id",
1_000L,
"_last_updated_sequence_number",
33L)),
record.copy(Map.of("id", 4L, "data", "d", "_row_id", 1_001L)),
record.copy(Map.of("id", 5L, "data", "e"))));
}
}
32 changes: 30 additions & 2 deletions data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@

import java.util.List;
import java.util.Map;
import org.apache.iceberg.MetadataColumns;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.variants.Variant;
Expand All @@ -31,12 +32,39 @@ public class DataTestHelpers {
private DataTestHelpers() {}

public static void assertEquals(Types.StructType struct, Record expected, Record actual) {
assertEquals(struct, expected, actual, null, -1);
}

public static void assertEquals(
Types.StructType struct,
Record expected,
Record actual,
Map<Integer, Object> idToConstant,
int pos) {
Types.StructType expectedType = expected.struct();
for (Types.NestedField field : struct.fields()) {
Types.NestedField expectedField = expectedType.field(field.fieldId());
Object expectedValue;
if (expectedField != null) {
assertEquals(
field.type(), expected.getField(expectedField.name()), actual.getField(field.name()));
int id = expectedField.fieldId();
if (id == MetadataColumns.ROW_ID.fieldId()) {
expectedValue = expected.getField(expectedField.name());
if (expectedValue == null && idToConstant != null) {
expectedValue = (Long) idToConstant.get(id) + pos;
}

} else if (id == MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId()) {
expectedValue = expected.getField(expectedField.name());
if (expectedValue == null && idToConstant != null) {
expectedValue = idToConstant.get(id);
}

} else {
expectedValue = expected.getField(expectedField.name());
}

assertEquals(field.type(), expectedValue, actual.getField(field.name()));

} else {
assertEquals(
field.type(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,10 @@
import org.apache.iceberg.data.DataTestHelpers;
import org.apache.iceberg.data.RandomGenericData;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.inmemory.InMemoryOutputFile;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.parquet.Parquet;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.types.Types;
Expand All @@ -61,6 +63,11 @@ protected boolean supportsTimestampNanos() {
return true;
}

@Override
protected boolean supportsRowLineage() {
return true;
}

@Override
protected void writeAndValidate(Schema schema) throws IOException {
writeAndValidate(schema, schema);
Expand All @@ -80,11 +87,10 @@ protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throw
private void writeAndValidate(Schema writeSchema, Schema expectedSchema, List<Record> expected)
throws IOException {

File testFile = File.createTempFile("junit", null, temp.toFile());
assertThat(testFile.delete()).isTrue();
OutputFile output = new InMemoryOutputFile();

try (FileAppender<Record> appender =
Parquet.write(Files.localOutput(testFile))
Parquet.write(output)
.schema(writeSchema)
.createWriterFunc(GenericParquetWriter::create)
.build()) {
Expand All @@ -93,30 +99,34 @@ private void writeAndValidate(Schema writeSchema, Schema expectedSchema, List<Re

List<Record> rows;
try (CloseableIterable<Record> reader =
Parquet.read(Files.localInput(testFile))
Parquet.read(output.toInputFile())
.project(expectedSchema)
.createReaderFunc(
fileSchema -> GenericParquetReaders.buildReader(expectedSchema, fileSchema))
fileSchema ->
GenericParquetReaders.buildReader(expectedSchema, fileSchema, ID_TO_CONSTANT))
.build()) {
rows = Lists.newArrayList(reader);
}

for (int i = 0; i < expected.size(); i += 1) {
DataTestHelpers.assertEquals(expectedSchema.asStruct(), expected.get(i), rows.get(i));
for (int pos = 0; pos < expected.size(); pos += 1) {
DataTestHelpers.assertEquals(
expectedSchema.asStruct(), expected.get(pos), rows.get(pos), ID_TO_CONSTANT, pos);
}

// test reuseContainers
try (CloseableIterable<Record> reader =
Parquet.read(Files.localInput(testFile))
Parquet.read(output.toInputFile())
.project(expectedSchema)
.reuseContainers()
.createReaderFunc(
fileSchema -> GenericParquetReaders.buildReader(expectedSchema, fileSchema))
fileSchema ->
GenericParquetReaders.buildReader(expectedSchema, fileSchema, ID_TO_CONSTANT))
.build()) {
int index = 0;
int pos = 0;
for (Record actualRecord : reader) {
DataTestHelpers.assertEquals(expectedSchema.asStruct(), expected.get(index), actualRecord);
index += 1;
DataTestHelpers.assertEquals(
expectedSchema.asStruct(), expected.get(pos), actualRecord, ID_TO_CONSTANT, pos);
pos += 1;
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.iceberg.MetadataColumns;
import org.apache.iceberg.Schema;
import org.apache.iceberg.parquet.ParquetSchemaUtil;
import org.apache.iceberg.parquet.ParquetValueReader;
Expand Down Expand Up @@ -77,7 +76,7 @@ protected ParquetValueReader<T> createReader(
}

protected abstract ParquetValueReader<T> createStructReader(
List<Type> types, List<ParquetValueReader<?>> fieldReaders, Types.StructType structType);
List<ParquetValueReader<?>> fieldReaders, Types.StructType structType);

protected abstract ParquetValueReader<?> fixedReader(ColumnDescriptor desc);

Expand Down Expand Up @@ -110,19 +109,17 @@ public ParquetValueReader<?> struct(
// the expected struct is ignored because nested fields are never found when the
List<ParquetValueReader<?>> newFields =
Lists.newArrayListWithExpectedSize(fieldReaders.size());
List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size());
List<Type> fields = struct.getFields();
for (int i = 0; i < fields.size(); i += 1) {
ParquetValueReader<?> fieldReader = fieldReaders.get(i);
if (fieldReader != null) {
Type fieldType = fields.get(i);
int fieldD = type().getMaxDefinitionLevel(path(fieldType.getName())) - 1;
newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReader));
types.add(fieldType);
}
}

return createStructReader(types, newFields, expected);
return createStructReader(newFields, expected);
}
}

Expand Down Expand Up @@ -225,10 +222,12 @@ public ParquetValueReader<?> message(
@Override
public ParquetValueReader<?> struct(
Types.StructType expected, GroupType struct, List<ParquetValueReader<?>> fieldReaders) {
if (null == expected) {
return createStructReader(ImmutableList.of(), null);
}

// match the expected struct's order
Map<Integer, ParquetValueReader<?>> readersById = Maps.newHashMap();
Map<Integer, Type> typesById = Maps.newHashMap();
Map<Integer, Integer> maxDefinitionLevelsById = Maps.newHashMap();
List<Type> fields = struct.getFields();
for (int i = 0; i < fields.size(); i += 1) {
ParquetValueReader<?> fieldReader = fieldReaders.get(i);
Expand All @@ -237,55 +236,37 @@ public ParquetValueReader<?> struct(
int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
int id = fieldType.getId().intValue();
readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReader));
typesById.put(id, fieldType);
if (idToConstant.containsKey(id)) {
maxDefinitionLevelsById.put(id, fieldD);
}
}
}

List<Types.NestedField> expectedFields =
expected != null ? expected.fields() : ImmutableList.of();
int constantDefinitionLevel = type.getMaxDefinitionLevel(currentPath());
List<Types.NestedField> expectedFields = expected.fields();
List<ParquetValueReader<?>> reorderedFields =
Lists.newArrayListWithExpectedSize(expectedFields.size());
List<Type> types = Lists.newArrayListWithExpectedSize(expectedFields.size());
// Defaulting to parent max definition level
int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath());

for (Types.NestedField field : expectedFields) {
int id = field.fieldId();
ParquetValueReader<?> reader = readersById.get(id);
if (idToConstant.containsKey(id)) {
// containsKey is used because the constant may be null
int fieldMaxDefinitionLevel =
maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel);
reorderedFields.add(
ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel));
types.add(null);
} else if (id == MetadataColumns.ROW_POSITION.fieldId()) {
reorderedFields.add(ParquetValueReaders.position());
types.add(null);
} else if (id == MetadataColumns.IS_DELETED.fieldId()) {
reorderedFields.add(ParquetValueReaders.constant(false));
types.add(null);
} else if (reader != null) {
reorderedFields.add(reader);
types.add(typesById.get(id));
} else if (field.initialDefault() != null) {
reorderedFields.add(
ParquetValueReaders.constant(
convertConstant(field.type(), field.initialDefault()),
maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel)));
types.add(typesById.get(id));
} else if (field.isOptional()) {
reorderedFields.add(ParquetValueReaders.nulls());
types.add(null);
} else {
throw new IllegalArgumentException(
String.format("Missing required field: %s", field.name()));
}
ParquetValueReader<?> reader =
ParquetValueReaders.replaceWithMetadataReader(
id, readersById.get(id), idToConstant, constantDefinitionLevel);
reorderedFields.add(defaultReader(field, reader, constantDefinitionLevel));
Copy link
Contributor Author

@rdblue rdblue Apr 18, 2025

Choose a reason for hiding this comment

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

The changes here were needed to reduce the complexity of the struct method that was failing checkstyle. I refactored into 2 components:

  • ParquetValueReaders.replaceWithMetadataReader handles metadata columns and constants
  • defaultReader handles defaults based on the schema; this requires convertConstant so it is not shared in ParquetValueReaders

I also simplified the constant handling that was introduced in #4627. That PR introduced a map with the max definition level for columns that are replaced with constants and defaults the definition level to the parent struct's value when there is no underlying column (for instance, _pos). This was over-complicated because the fields don't need a column-specific DL. If the parent struct is non-null then the constant should be added to it, so the DL can always be the parent's DL.

}

return createStructReader(reorderedFields, expected);
}

private ParquetValueReader<?> defaultReader(
Types.NestedField field, ParquetValueReader<?> reader, int constantDL) {
if (reader != null) {
return reader;
} else if (field.initialDefault() != null) {
return ParquetValueReaders.constant(
convertConstant(field.type(), field.initialDefault()), constantDL);
} else if (field.isOptional()) {
return ParquetValueReaders.nulls();
}

return createStructReader(types, reorderedFields, expected);
throw new IllegalArgumentException(String.format("Missing required field: %s", field.name()));
}

@Override
Expand Down
Loading