Skip to content

Commit ddf18ab

Browse files
committed
Fix struct fieldid if missing in fileschema, read from expected(which is projected schema.
1 parent f38c341 commit ddf18ab

File tree

2 files changed

+148
-4
lines changed

2 files changed

+148
-4
lines changed

parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -198,11 +198,12 @@ private static <T> T visitField(
198198
private static <T> List<T> visitFields(
199199
Types.StructType struct, GroupType group, TypeWithSchemaVisitor<T> visitor) {
200200
List<T> results = Lists.newArrayListWithExpectedSize(group.getFieldCount());
201+
int i =0;
201202
for (Type field : group.getFields()) {
202-
int id = -1;
203-
if (field.getId() != null) {
204-
id = field.getId().intValue();
205-
}
203+
int id = field.getId() != null
204+
? field.getId().intValue()
205+
: (struct != null) ? struct.fields().get(i).fieldId() : -1;
206+
i++;
206207
Types.NestedField iField = (struct != null && id >= 0) ? struct.field(id) : null;
207208
results.add(visitField(iField, field, visitor));
208209
}

parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java

Lines changed: 143 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,9 @@
4444
import org.apache.iceberg.MetricsConfig;
4545
import org.apache.iceberg.Schema;
4646
import org.apache.iceberg.avro.AvroSchemaUtil;
47+
import org.apache.iceberg.data.Record;
48+
import org.apache.iceberg.data.parquet.GenericParquetReaders;
49+
import org.apache.iceberg.io.CloseableIterable;
4750
import org.apache.iceberg.io.InputFile;
4851
import org.apache.iceberg.relocated.com.google.common.base.Strings;
4952
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
@@ -259,4 +262,144 @@ private Pair<File, Long> generateFile(
259262
records.toArray(new GenericData.Record[] {}));
260263
return Pair.of(file, size);
261264
}
265+
266+
@Test
267+
public void testReadNestedStructWithoutId() throws IOException {
268+
Schema icebergSchema =
269+
new Schema(
270+
Types.NestedField.required(
271+
1,
272+
"outer_struct",
273+
Types.StructType.of(
274+
Types.NestedField.optional(
275+
2,
276+
"middle_struct",
277+
Types.StructType.of(
278+
Types.NestedField.optional(
279+
3,
280+
"inner_struct",
281+
Types.StructType.of(
282+
Types.NestedField.optional(
283+
4, "value_field", Types.StringType.get()))))))));
284+
285+
// Create Avro schema without IDs.
286+
org.apache.avro.Schema avroSchema = createAvroSchemaWithoutIds();
287+
288+
// Write test data to Parquet file.
289+
File file = createTempFile(temp);
290+
writeParquetFile(file, avroSchema);
291+
292+
// Read and verify the data.
293+
try (CloseableIterable<Record> reader =
294+
Parquet.read(Files.localInput(file))
295+
.project(icebergSchema)
296+
.createReaderFunc(
297+
fileSchema -> GenericParquetReaders.buildReader(icebergSchema, fileSchema))
298+
.build()) {
299+
300+
org.apache.iceberg.data.Record readRecord = Iterables.getOnlyElement(reader);
301+
verifyNestedStructData(readRecord);
302+
}
303+
}
304+
305+
private org.apache.avro.Schema createAvroSchemaWithoutIds() {
306+
org.apache.avro.Schema innerStructSchema =
307+
org.apache.avro.Schema.createRecord("inner_struct_type", null, null, false);
308+
innerStructSchema.setFields(
309+
List.of(
310+
new org.apache.avro.Schema.Field(
311+
"value_field",
312+
org.apache.avro.Schema.createUnion(
313+
List.of(
314+
org.apache.avro.Schema.create(org.apache.avro.Schema.Type.NULL),
315+
org.apache.avro.Schema.create(org.apache.avro.Schema.Type.STRING))),
316+
null,
317+
org.apache.avro.JsonProperties.NULL_VALUE)));
318+
319+
org.apache.avro.Schema middleStructSchema =
320+
org.apache.avro.Schema.createRecord("middle_struct_type", null, null, false);
321+
middleStructSchema.setFields(
322+
List.of(
323+
new org.apache.avro.Schema.Field(
324+
"inner_struct",
325+
org.apache.avro.Schema.createUnion(
326+
List.of(
327+
org.apache.avro.Schema.create(org.apache.avro.Schema.Type.NULL),
328+
innerStructSchema)),
329+
null,
330+
org.apache.avro.JsonProperties.NULL_VALUE)));
331+
332+
org.apache.avro.Schema outerStructSchema =
333+
org.apache.avro.Schema.createRecord("outer_struct_type", null, null, false);
334+
outerStructSchema.setFields(
335+
List.of(
336+
new org.apache.avro.Schema.Field(
337+
"middle_struct",
338+
org.apache.avro.Schema.createUnion(
339+
List.of(
340+
org.apache.avro.Schema.create(org.apache.avro.Schema.Type.NULL),
341+
middleStructSchema)),
342+
null,
343+
org.apache.avro.JsonProperties.NULL_VALUE)));
344+
345+
org.apache.avro.Schema recordSchema =
346+
org.apache.avro.Schema.createRecord("test_record", null, null, false);
347+
recordSchema.setFields(
348+
List.of(new org.apache.avro.Schema.Field("outer_struct", outerStructSchema, null, null)));
349+
350+
return recordSchema;
351+
}
352+
353+
private void writeParquetFile(File file, org.apache.avro.Schema avroSchema) throws IOException {
354+
// Create test data.
355+
GenericData.Record record = createNestedRecord(avroSchema);
356+
357+
// Write to Parquet file.
358+
try (ParquetWriter<GenericRecord> writer =
359+
AvroParquetWriter.<GenericRecord>builder(new org.apache.hadoop.fs.Path(file.toURI()))
360+
.withSchema(avroSchema)
361+
.withDataModel(GenericData.get())
362+
.build()) {
363+
writer.write(record);
364+
}
365+
}
366+
367+
private GenericData.Record createNestedRecord(org.apache.avro.Schema avroSchema) {
368+
org.apache.avro.Schema outerSchema = avroSchema.getField("outer_struct").schema();
369+
org.apache.avro.Schema middleSchema =
370+
outerSchema.getField("middle_struct").schema().getTypes().get(1);
371+
org.apache.avro.Schema innerSchema =
372+
middleSchema.getField("inner_struct").schema().getTypes().get(1);
373+
374+
GenericRecordBuilder innerBuilder = new GenericRecordBuilder(innerSchema);
375+
innerBuilder.set("value_field", "test_value");
376+
377+
GenericRecordBuilder middleBuilder = new GenericRecordBuilder(middleSchema);
378+
middleBuilder.set("inner_struct", innerBuilder.build());
379+
380+
GenericRecordBuilder outerBuilder = new GenericRecordBuilder(outerSchema);
381+
outerBuilder.set("middle_struct", middleBuilder.build());
382+
383+
GenericRecordBuilder recordBuilder = new GenericRecordBuilder(avroSchema);
384+
recordBuilder.set("outer_struct", outerBuilder.build());
385+
386+
return recordBuilder.build();
387+
}
388+
389+
private void verifyNestedStructData(org.apache.iceberg.data.Record record) {
390+
org.apache.iceberg.data.Record outerStruct = (org.apache.iceberg.data.Record) record.get(0);
391+
assertThat(outerStruct).isNotNull().withFailMessage("Outer struct should not be null");
392+
393+
org.apache.iceberg.data.Record middleStruct =
394+
(org.apache.iceberg.data.Record) outerStruct.get(0);
395+
assertThat(middleStruct).isNotNull().withFailMessage("Middle struct should not be null");
396+
397+
org.apache.iceberg.data.Record innerStruct =
398+
(org.apache.iceberg.data.Record) middleStruct.get(0);
399+
assertThat(innerStruct).isNotNull().withFailMessage("Inner struct should not be null");
400+
401+
assertThat(innerStruct.get(0).toString())
402+
.isEqualTo("test_value")
403+
.withFailMessage("Inner value field should match expected value");
404+
}
262405
}

0 commit comments

Comments
 (0)