aihuaxu commented on code in PR #12512: URL: https://github.com/apache/iceberg/pull/12512#discussion_r2006560551
########## parquet/src/main/java/org/apache/iceberg/parquet/ParquetVariantReaders.java: ########## @@ -332,6 +346,57 @@ public void setPageSource(PageReadStore pageStore) { } } + private static class ListReader implements VariantValueReader { Review Comment: I changed from `ArrayReader` to `ListReader` because in the following writer implementation, I need implement `ListWriter` to handle the list and then have a `ArrayValueWriter` on top of `ListWriter` to combine with value. I name it as `ListReader` to be consistent so `shredded(ListReader)` will be actual `ArrayReader`. ########## parquet/src/test/java/org/apache/iceberg/parquet/TestVariantReaders.java: ########## @@ -879,6 +891,270 @@ public void testMixedRecords() throws IOException { VariantTestUtil.assertEqual(expectedThree, actualThreeVariant.value()); } + @Test + public void testSimpleArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(shreddedType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = elements(shreddedType, List.of("comedy", "drama")); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + Record actual = writeAndRead(parquetSchema, row); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + ValueArray expectedArray = Variants.array(); + expectedArray.add(Variants.of("comedy")); + expectedArray.add(Variants.of("drama")); + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expectedArray, actualVariant.value()); + } + + @Test + public void testNullArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(shreddedType)); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord var = + record( + variantType, + Map.of( + "metadata", + VariantTestUtil.emptyMetadata(), + "value", + serialize(Variants.ofNull()))); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + Record actual = writeAndRead(parquetSchema, row); + + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(Variants.ofNull(), actualVariant.value()); + } + + @Test + public void testEmptyArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(shreddedType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = List.of(); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + Record actual = writeAndRead(parquetSchema, row); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant = (Variant) actual.getField("var"); + assertThat(actualVariant.value().type()).isEqualTo(PhysicalType.ARRAY); + assertThat(actualVariant.value().asArray().numElements()).isEqualTo(0); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + } + + @Test + public void testArrayWithNull() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(shreddedType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = elements(shreddedType, Lists.newArrayList("comedy", null, "drama")); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + Record actual = writeAndRead(parquetSchema, row); + + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant = (Variant) actual.getField("var"); + assertThat(actualVariant.value().type()).isEqualTo(PhysicalType.ARRAY); + assertThat(actualVariant.value().asArray().numElements()).isEqualTo(3); + ValueArray expectedArray = Variants.array(); + expectedArray.add(Variants.of("comedy")); + expectedArray.add(Variants.ofNull()); + expectedArray.add(Variants.of("drama")); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expectedArray, actualVariant.value()); + } + + @Test + public void testNestedArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType innerListType = list(shreddedType); + GroupType variantType = variant("var", 2, list(innerListType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> inner1 = elements(shreddedType, List.of("comedy", "drama")); + List<GenericRecord> inner2 = elements(shreddedType, List.of()); + List<GenericRecord> outer1 = elements(innerListType, List.of(inner1, inner2)); + GenericRecord var = + record( + variantType, + Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", outer1)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + Record actual = writeAndRead(parquetSchema, row); + + // Verify + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + ValueArray expectedArray = Variants.array(); + ValueArray expectedInner1 = Variants.array(); + expectedInner1.add(Variants.of("comedy")); + expectedInner1.add(Variants.of("drama")); + ValueArray expectedInner2 = Variants.array(); + expectedArray.add(expectedInner1); + expectedArray.add(expectedInner2); + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expectedArray, actualVariant.value()); Review Comment: Followed you suggestion to move expected creation above. assertion is shorter now. ########## parquet/src/test/java/org/apache/iceberg/parquet/TestVariantReaders.java: ########## @@ -879,6 +891,270 @@ public void testMixedRecords() throws IOException { VariantTestUtil.assertEqual(expectedThree, actualThreeVariant.value()); } + @Test + public void testSimpleArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(shreddedType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = elements(shreddedType, List.of("comedy", "drama")); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + Record actual = writeAndRead(parquetSchema, row); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + ValueArray expectedArray = Variants.array(); + expectedArray.add(Variants.of("comedy")); + expectedArray.add(Variants.of("drama")); + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expectedArray, actualVariant.value()); + } + + @Test + public void testNullArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(shreddedType)); + MessageType parquetSchema = parquetSchema(variantType); + + GenericRecord var = + record( + variantType, + Map.of( + "metadata", + VariantTestUtil.emptyMetadata(), + "value", + serialize(Variants.ofNull()))); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + Record actual = writeAndRead(parquetSchema, row); + + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant = (Variant) actual.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(Variants.ofNull(), actualVariant.value()); + } + + @Test + public void testEmptyArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(shreddedType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = List.of(); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + Record actual = writeAndRead(parquetSchema, row); + assertThat(actual.getField("id")).isEqualTo(1); + assertThat(actual.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant = (Variant) actual.getField("var"); + assertThat(actualVariant.value().type()).isEqualTo(PhysicalType.ARRAY); + assertThat(actualVariant.value().asArray().numElements()).isEqualTo(0); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + } + + @Test + public void testArrayWithNull() throws IOException { Review Comment: The case that both `value` and `typed_value` are null for the element is invalid case and we don't define that in the spec. We are getting null. ########## parquet/src/main/java/org/apache/iceberg/parquet/ParquetVariantReaders.java: ########## @@ -95,6 +96,19 @@ public static VariantValueReader objects( fieldReaders); } + public static VariantValueReader array( + int valueDefinitionLevel, + ParquetValueReader<?> valueReader, + int typedDefinitionLevel, + int repeatedDefinitionLevel, + int repeatedRepetitionLevel, + ParquetValueReader<?> elementReader) { + VariantValueReader typedReader = + new ListReader( Review Comment: Yeah. That's cleaner. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org