rdblue commented on code in PR #12512: URL: https://github.com/apache/iceberg/pull/12512#discussion_r2059371134
########## parquet/src/test/java/org/apache/iceberg/parquet/TestVariantReaders.java: ########## @@ -879,6 +896,405 @@ public void testMixedRecords() throws IOException { VariantTestUtil.assertEqual(expectedThree, actualThreeVariant.value()); } + @Test + public void testSimpleArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType elementType = element(shreddedType); + GroupType variantType = variant("var", 2, list(elementType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = + elements(elementType, List.of(Variants.of("comedy"), Variants.of("drama"))); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + ValueArray expectedArray = Variants.array(); + expectedArray.add(Variants.of("comedy")); + expectedArray.add(Variants.of("drama")); + + 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(expectedArray, actualVariant.value()); + } + + @Test + public void testNullArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(element(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(element(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 elementType = element(shreddedType); + GroupType variantType = variant("var", 2, list(elementType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = + elements( + elementType, + Lists.newArrayList( + List.of(Variants.of("comedy"), Variants.ofNull(), Variants.of("drama")))); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + ValueArray expectedArray = Variants.array(); + expectedArray.add(Variants.of("comedy")); + expectedArray.add(Variants.ofNull()); + expectedArray.add(Variants.of("drama")); + + 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); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expectedArray, actualVariant.value()); + } + + @Test + public void testNestedArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType elementType = element(shreddedType); + GroupType outerElementType = element(list(elementType)); + GroupType variantType = variant("var", 2, list(outerElementType)); + MessageType parquetSchema = parquetSchema(variantType); + + ValueArray inner1 = Variants.array(); + inner1.add(Variants.of("comedy")); + inner1.add(Variants.of("drama")); + + List<GenericRecord> outer1 = elements(outerElementType, List.of(inner1, Variants.array())); + GenericRecord var = + record( + variantType, + Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", outer1)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + 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); + + Record actual = writeAndRead(parquetSchema, row); + + // Verify + 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(expectedArray, actualVariant.value()); + } + + @Test + public void testArrayWithNestedObject() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType shreddedFields = objectFields(fieldA, fieldB); + GroupType elementType = element(shreddedFields); + GroupType listType = list(elementType); + GroupType fieldC = field("c", listType); + GroupType objectFields = objectFields(fieldC); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + // Row 1 + ByteBuffer shreddedBuffer1 = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("a", Variants.of(1), "b", Variants.of("comedy"))); + VariantValue shredded1 = Variants.value(TEST_METADATA, shreddedBuffer1); + ByteBuffer shreddedBuffer2 = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("a", Variants.of(2), "b", Variants.of("drama"))); + VariantValue shredded2 = Variants.value(TEST_METADATA, shreddedBuffer2); + + List<GenericRecord> arr1 = elements(elementType, List.of(shredded1, shredded2)); + GenericRecord element1 = record(fieldC, Map.of("typed_value", arr1)); + GenericRecord c1 = record(objectFields, Map.of("c", element1)); + GenericRecord var1 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", c1)); + GenericRecord row1 = record(parquetSchema, Map.of("id", 1, "var", var1)); + + ShreddedObject expected1 = Variants.object(TEST_METADATA); + ValueArray expectedArray1 = Variants.array(); + ShreddedObject expectedElement1 = Variants.object(TEST_METADATA); + expectedElement1.put("a", Variants.of(1)); + expectedElement1.put("b", Variants.of("comedy")); + expectedArray1.add(expectedElement1); + ShreddedObject expectedElement2 = Variants.object(TEST_METADATA); + expectedElement2.put("a", Variants.of(2)); + expectedElement2.put("b", Variants.of("drama")); + expectedArray1.add(expectedElement2); + expected1.put("c", expectedArray1); + + // Row 2 + ByteBuffer shreddedBuffer3 = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("a", Variants.of(3), "b", Variants.of("action"))); + VariantValue shredded3 = Variants.value(TEST_METADATA, shreddedBuffer3); + ByteBuffer shreddedBuffer4 = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("a", Variants.of(4), "b", Variants.of("horror"))); + VariantValue shredded4 = Variants.value(TEST_METADATA, shreddedBuffer4); + List<GenericRecord> arr2 = elements(elementType, List.of(shredded3, shredded4)); + GenericRecord element2 = record(fieldC, Map.of("typed_value", arr2)); + GenericRecord c2 = record(objectFields, Map.of("c", element2)); + GenericRecord var2 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", c2)); + GenericRecord row2 = record(parquetSchema, Map.of("id", 2, "var", var2)); + + ShreddedObject expected2 = Variants.object(TEST_METADATA); + ValueArray expectedArray2 = Variants.array(); + ShreddedObject expectedElement3 = Variants.object(TEST_METADATA); + expectedElement3.put("a", Variants.of(3)); + expectedElement3.put("b", Variants.of("action")); + expectedArray2.add(expectedElement3); + ShreddedObject expectedElement4 = Variants.object(TEST_METADATA); + expectedElement4.put("a", Variants.of(4)); + expectedElement4.put("b", Variants.of("horror")); + expectedArray2.add(expectedElement4); + expected2.put("c", expectedArray2); + + // verify + List<Record> actual = writeAndRead(parquetSchema, List.of(row1, row2)); + Record actual1 = actual.get(0); + assertThat(actual1.getField("id")).isEqualTo(1); + assertThat(actual1.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant1 = (Variant) actual1.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant1.metadata()); + VariantTestUtil.assertEqual(expected1, actualVariant1.value()); + + Record actual2 = actual.get(1); + assertThat(actual2.getField("id")).isEqualTo(2); + assertThat(actual2.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant2 = (Variant) actual2.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant2.metadata()); + VariantTestUtil.assertEqual(expected2, actualVariant2.value()); + } + + @Test + public void testArrayWithNonArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType elementType = element(shreddedType); + GroupType variantType = variant("var", 2, list(elementType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr1 = + elements(elementType, List.of(Variants.of("comedy"), Variants.of("drama"))); + GenericRecord var1 = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr1)); + GenericRecord row1 = record(parquetSchema, Map.of("id", 1, "var", var1)); + + ValueArray expectedArray1 = Variants.array(); + expectedArray1.add(Variants.of("comedy")); + expectedArray1.add(Variants.of("drama")); + + GenericRecord var2 = + record( + variantType, + Map.of( + "metadata", VariantTestUtil.emptyMetadata(), "value", serialize(Variants.of(34)))); + GenericRecord row2 = record(parquetSchema, Map.of("id", 2, "var", var2)); + + VariantValue expectedValue2 = Variants.of(PhysicalType.INT32, 34); + + GenericRecord var3 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "value", TEST_OBJECT_BUFFER)); + GenericRecord row3 = record(parquetSchema, Map.of("id", 3, "var", var3)); + + ShreddedObject expectedObject3 = Variants.object(TEST_METADATA); + expectedObject3.put("a", Variants.ofNull()); + expectedObject3.put("d", Variants.of("iceberg")); + + // Test array is read properly after a non-array + List<GenericRecord> arr4 = + elements(elementType, List.of(Variants.of("action"), Variants.of("horror"))); + GenericRecord var4 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", arr4)); + GenericRecord row4 = record(parquetSchema, Map.of("id", 4, "var", var4)); + + ValueArray expectedArray4 = Variants.array(); + expectedArray4.add(Variants.of("action")); + expectedArray4.add(Variants.of("horror")); + + List<Record> actual = writeAndRead(parquetSchema, List.of(row1, row2, row3, row4)); + + // Verify + Record actual1 = actual.get(0); + assertThat(actual1.getField("id")).isEqualTo(1); + assertThat(actual1.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant1 = (Variant) actual1.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant1.metadata()); + VariantTestUtil.assertEqual(expectedArray1, actualVariant1.value()); + + Record actual2 = actual.get(1); + assertThat(actual2.getField("id")).isEqualTo(2); + assertThat(actual2.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant2 = (Variant) actual2.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant2.metadata()); + VariantTestUtil.assertEqual(expectedValue2, actualVariant2.value()); + + Record actual3 = actual.get(2); + assertThat(actual3.getField("id")).isEqualTo(3); + assertThat(actual3.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant3 = (Variant) actual3.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant3.metadata()); + VariantTestUtil.assertEqual(expectedObject3, actualVariant3.value()); + + Record actual4 = actual.get(3); + assertThat(actual4.getField("id")).isEqualTo(4); + assertThat(actual4.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant4 = (Variant) actual4.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant4.metadata()); + VariantTestUtil.assertEqual(expectedArray4, actualVariant4.value()); + } + + @Test + public void testArrayMissingValueColumn() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType elementType = element(shreddedType); Review Comment: This isn't testing an array missing the value column because `element` creates the inner `value` column. I think this is actually just exercising the value/typed_value handling in `ShreddedVariantReader`. ########## parquet/src/test/java/org/apache/iceberg/parquet/TestVariantReaders.java: ########## @@ -879,6 +896,405 @@ public void testMixedRecords() throws IOException { VariantTestUtil.assertEqual(expectedThree, actualThreeVariant.value()); } + @Test + public void testSimpleArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType elementType = element(shreddedType); + GroupType variantType = variant("var", 2, list(elementType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = + elements(elementType, List.of(Variants.of("comedy"), Variants.of("drama"))); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + ValueArray expectedArray = Variants.array(); + expectedArray.add(Variants.of("comedy")); + expectedArray.add(Variants.of("drama")); + + 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(expectedArray, actualVariant.value()); + } + + @Test + public void testNullArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType variantType = variant("var", 2, list(element(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(element(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 elementType = element(shreddedType); + GroupType variantType = variant("var", 2, list(elementType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr = + elements( + elementType, + Lists.newArrayList( + List.of(Variants.of("comedy"), Variants.ofNull(), Variants.of("drama")))); + GenericRecord var = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + ValueArray expectedArray = Variants.array(); + expectedArray.add(Variants.of("comedy")); + expectedArray.add(Variants.ofNull()); + expectedArray.add(Variants.of("drama")); + + 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); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant.metadata()); + VariantTestUtil.assertEqual(expectedArray, actualVariant.value()); + } + + @Test + public void testNestedArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType elementType = element(shreddedType); + GroupType outerElementType = element(list(elementType)); + GroupType variantType = variant("var", 2, list(outerElementType)); + MessageType parquetSchema = parquetSchema(variantType); + + ValueArray inner1 = Variants.array(); + inner1.add(Variants.of("comedy")); + inner1.add(Variants.of("drama")); + + List<GenericRecord> outer1 = elements(outerElementType, List.of(inner1, Variants.array())); + GenericRecord var = + record( + variantType, + Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", outer1)); + GenericRecord row = record(parquetSchema, Map.of("id", 1, "var", var)); + + 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); + + Record actual = writeAndRead(parquetSchema, row); + + // Verify + 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(expectedArray, actualVariant.value()); + } + + @Test + public void testArrayWithNestedObject() throws IOException { + GroupType fieldA = field("a", shreddedPrimitive(PrimitiveTypeName.INT32)); + GroupType fieldB = field("b", shreddedPrimitive(PrimitiveTypeName.BINARY, STRING)); + GroupType shreddedFields = objectFields(fieldA, fieldB); + GroupType elementType = element(shreddedFields); + GroupType listType = list(elementType); + GroupType fieldC = field("c", listType); + GroupType objectFields = objectFields(fieldC); + GroupType variantType = variant("var", 2, objectFields); + MessageType parquetSchema = parquetSchema(variantType); + + // Row 1 + ByteBuffer shreddedBuffer1 = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("a", Variants.of(1), "b", Variants.of("comedy"))); + VariantValue shredded1 = Variants.value(TEST_METADATA, shreddedBuffer1); + ByteBuffer shreddedBuffer2 = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("a", Variants.of(2), "b", Variants.of("drama"))); + VariantValue shredded2 = Variants.value(TEST_METADATA, shreddedBuffer2); + + List<GenericRecord> arr1 = elements(elementType, List.of(shredded1, shredded2)); + GenericRecord element1 = record(fieldC, Map.of("typed_value", arr1)); + GenericRecord c1 = record(objectFields, Map.of("c", element1)); + GenericRecord var1 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", c1)); + GenericRecord row1 = record(parquetSchema, Map.of("id", 1, "var", var1)); + + ShreddedObject expected1 = Variants.object(TEST_METADATA); + ValueArray expectedArray1 = Variants.array(); + ShreddedObject expectedElement1 = Variants.object(TEST_METADATA); + expectedElement1.put("a", Variants.of(1)); + expectedElement1.put("b", Variants.of("comedy")); + expectedArray1.add(expectedElement1); + ShreddedObject expectedElement2 = Variants.object(TEST_METADATA); + expectedElement2.put("a", Variants.of(2)); + expectedElement2.put("b", Variants.of("drama")); + expectedArray1.add(expectedElement2); + expected1.put("c", expectedArray1); + + // Row 2 + ByteBuffer shreddedBuffer3 = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("a", Variants.of(3), "b", Variants.of("action"))); + VariantValue shredded3 = Variants.value(TEST_METADATA, shreddedBuffer3); + ByteBuffer shreddedBuffer4 = + VariantTestUtil.createObject( + TEST_METADATA_BUFFER, Map.of("a", Variants.of(4), "b", Variants.of("horror"))); + VariantValue shredded4 = Variants.value(TEST_METADATA, shreddedBuffer4); + List<GenericRecord> arr2 = elements(elementType, List.of(shredded3, shredded4)); + GenericRecord element2 = record(fieldC, Map.of("typed_value", arr2)); + GenericRecord c2 = record(objectFields, Map.of("c", element2)); + GenericRecord var2 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", c2)); + GenericRecord row2 = record(parquetSchema, Map.of("id", 2, "var", var2)); + + ShreddedObject expected2 = Variants.object(TEST_METADATA); + ValueArray expectedArray2 = Variants.array(); + ShreddedObject expectedElement3 = Variants.object(TEST_METADATA); + expectedElement3.put("a", Variants.of(3)); + expectedElement3.put("b", Variants.of("action")); + expectedArray2.add(expectedElement3); + ShreddedObject expectedElement4 = Variants.object(TEST_METADATA); + expectedElement4.put("a", Variants.of(4)); + expectedElement4.put("b", Variants.of("horror")); + expectedArray2.add(expectedElement4); + expected2.put("c", expectedArray2); + + // verify + List<Record> actual = writeAndRead(parquetSchema, List.of(row1, row2)); + Record actual1 = actual.get(0); + assertThat(actual1.getField("id")).isEqualTo(1); + assertThat(actual1.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant1 = (Variant) actual1.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant1.metadata()); + VariantTestUtil.assertEqual(expected1, actualVariant1.value()); + + Record actual2 = actual.get(1); + assertThat(actual2.getField("id")).isEqualTo(2); + assertThat(actual2.getField("var")).isInstanceOf(Variant.class); + + Variant actualVariant2 = (Variant) actual2.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant2.metadata()); + VariantTestUtil.assertEqual(expected2, actualVariant2.value()); + } + + @Test + public void testArrayWithNonArray() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType elementType = element(shreddedType); + GroupType variantType = variant("var", 2, list(elementType)); + MessageType parquetSchema = parquetSchema(variantType); + + List<GenericRecord> arr1 = + elements(elementType, List.of(Variants.of("comedy"), Variants.of("drama"))); + GenericRecord var1 = + record( + variantType, Map.of("metadata", VariantTestUtil.emptyMetadata(), "typed_value", arr1)); + GenericRecord row1 = record(parquetSchema, Map.of("id", 1, "var", var1)); + + ValueArray expectedArray1 = Variants.array(); + expectedArray1.add(Variants.of("comedy")); + expectedArray1.add(Variants.of("drama")); + + GenericRecord var2 = + record( + variantType, + Map.of( + "metadata", VariantTestUtil.emptyMetadata(), "value", serialize(Variants.of(34)))); + GenericRecord row2 = record(parquetSchema, Map.of("id", 2, "var", var2)); + + VariantValue expectedValue2 = Variants.of(PhysicalType.INT32, 34); + + GenericRecord var3 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "value", TEST_OBJECT_BUFFER)); + GenericRecord row3 = record(parquetSchema, Map.of("id", 3, "var", var3)); + + ShreddedObject expectedObject3 = Variants.object(TEST_METADATA); + expectedObject3.put("a", Variants.ofNull()); + expectedObject3.put("d", Variants.of("iceberg")); + + // Test array is read properly after a non-array + List<GenericRecord> arr4 = + elements(elementType, List.of(Variants.of("action"), Variants.of("horror"))); + GenericRecord var4 = + record(variantType, Map.of("metadata", TEST_METADATA_BUFFER, "typed_value", arr4)); + GenericRecord row4 = record(parquetSchema, Map.of("id", 4, "var", var4)); + + ValueArray expectedArray4 = Variants.array(); + expectedArray4.add(Variants.of("action")); + expectedArray4.add(Variants.of("horror")); + + List<Record> actual = writeAndRead(parquetSchema, List.of(row1, row2, row3, row4)); + + // Verify + Record actual1 = actual.get(0); + assertThat(actual1.getField("id")).isEqualTo(1); + assertThat(actual1.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant1 = (Variant) actual1.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant1.metadata()); + VariantTestUtil.assertEqual(expectedArray1, actualVariant1.value()); + + Record actual2 = actual.get(1); + assertThat(actual2.getField("id")).isEqualTo(2); + assertThat(actual2.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant2 = (Variant) actual2.getField("var"); + VariantTestUtil.assertEqual(EMPTY_METADATA, actualVariant2.metadata()); + VariantTestUtil.assertEqual(expectedValue2, actualVariant2.value()); + + Record actual3 = actual.get(2); + assertThat(actual3.getField("id")).isEqualTo(3); + assertThat(actual3.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant3 = (Variant) actual3.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant3.metadata()); + VariantTestUtil.assertEqual(expectedObject3, actualVariant3.value()); + + Record actual4 = actual.get(3); + assertThat(actual4.getField("id")).isEqualTo(4); + assertThat(actual4.getField("var")).isInstanceOf(Variant.class); + Variant actualVariant4 = (Variant) actual4.getField("var"); + VariantTestUtil.assertEqual(TEST_METADATA, actualVariant4.metadata()); + VariantTestUtil.assertEqual(expectedArray4, actualVariant4.value()); + } + + @Test + public void testArrayMissingValueColumn() throws IOException { + Type shreddedType = shreddedPrimitive(PrimitiveTypeName.BINARY, STRING); + GroupType elementType = element(shreddedType); Review Comment: This isn't testing an array missing the value column because `element` creates the inner `value` column. I think this is actually just exercising the value/typed_value handling in `ShreddedVariantReader`, which is tested elsewhere. -- 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