rdblue commented on code in PR #15087:
URL: https://github.com/apache/iceberg/pull/15087#discussion_r2709915918


##########
parquet/src/test/java/org/apache/iceberg/parquet/TestVariantWriters.java:
##########
@@ -280,4 +283,61 @@ private static ValueArray array(VariantValue... values) {
 
     return arr;
   }
+
+  @Test
+  public void testPartialShreddingWithShreddedObject() throws IOException {
+    // Test for issue #15086: partial shredding with ShreddedObject created 
using put()
+    // Create a ShreddedObject with multiple fields, then partially shred it
+    VariantMetadata metadata = Variants.metadata("id", "name", "city");
+
+    // Create objects using ShreddedObject.put() instead of serialized buffers
+    List<Record> records = Lists.newArrayList();
+    for (int i = 0; i < 3; i++) {
+      ShreddedObject obj = Variants.object(metadata);
+      obj.put("id", Variants.of(1000L + i));
+      obj.put("name", Variants.of("user_" + i));
+      obj.put("city", Variants.of("city_" + i));
+
+      Variant variant = Variant.of(metadata, obj);
+      Record record = RECORD.copy("id", i, "var", variant);
+      records.add(record);
+    }
+
+    // Shredding function that only shreds the "id" field
+    VariantShreddingFunction partialShredding =
+        (id, name) ->
+            org.apache.parquet.schema.Types.optionalGroup()
+                .addField(
+                    org.apache.parquet.schema.Types.optionalGroup()
+                        .addField(
+                            org.apache.parquet.schema.Types.optional(
+                                    PrimitiveType.PrimitiveTypeName.BINARY)
+                                .named("value"))
+                        .addField(
+                            org.apache.parquet.schema.Types.optional(
+                                    PrimitiveType.PrimitiveTypeName.INT64)
+                                .named("typed_value"))
+                        .named("id"))
+                .named("typed_value");
+
+    // Write and read back
+    List<Record> actual = writeAndRead(partialShredding, records);

Review Comment:
   I looked into this a bit more and ended up producing a test for 
`TestVariantReaders` that reproduces the problem. Here's my full diff of the 
test and the fix:
   
   ```diff
   diff --git 
a/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java 
b/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java
   index c1fba073cf..5d2bfc48fa 100644
   --- a/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java
   +++ b/core/src/main/java/org/apache/iceberg/variants/ShreddedObject.java
   @@ -153,12 +153,12 @@ public class ShreddedObject implements VariantObject {
        private SerializationState(
            VariantMetadata metadata,
            VariantObject unshredded,
   -        Map<String, VariantValue> shreddedFields,
   +        Map<String, VariantValue> shredded,
            Set<String> removedFields) {
          this.metadata = metadata;
          // field ID size is the size needed to store the largest field ID in 
the data
          this.fieldIdSize = VariantUtil.sizeOf(metadata.dictionarySize());
   -      this.shreddedFields = Maps.newHashMap(shreddedFields);
   +      this.shreddedFields = Maps.newHashMap(shredded);
    
          int totalDataSize = 0;
          // get the unshredded field names and values as byte buffers
   diff --git 
a/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java 
b/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java
   index 1ebb433a54..0cb043d420 100644
   --- a/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java
   +++ b/core/src/test/java/org/apache/iceberg/variants/TestShreddedObject.java
   @@ -213,6 +213,28 @@ public class TestShreddedObject {
            .isEqualTo(DateTimeUtil.isoDateToDays("2024-10-12"));
      }
    
   +  @Test
   +  public void 
testPartiallyShreddedUnserializedObjectSerializationMinimalBuffer() {
   +    ShreddedObject partial = createUnserializedObject(FIELDS);
   +    VariantMetadata metadata = partial.metadata();
   +
   +    // replace field c with a new value
   +    partial.put("c", Variants.ofIsoDate("2024-10-12"));
   +    partial.remove("b");
   +
   +    VariantValue value = roundTripMinimalBuffer(partial, metadata);
   +
   +    assertThat(value).isInstanceOf(SerializedObject.class);
   +    SerializedObject actual = (SerializedObject) value;
   +
   +    assertThat(actual.get("a")).isInstanceOf(VariantPrimitive.class);
   +    assertThat(actual.get("a").asPrimitive().get()).isEqualTo(34);
   +    assertThat(actual.get("c")).isInstanceOf(VariantPrimitive.class);
   +    assertThat(actual.get("c").type()).isEqualTo(PhysicalType.DATE);
   +    assertThat(actual.get("c").asPrimitive().get())
   +        .isEqualTo(DateTimeUtil.isoDateToDays("2024-10-12"));
   +  }
   +
      @Test
      public void testPartiallyShreddedObjectSerializationLargeBuffer() {
        ShreddedObject partial = createUnshreddedObject(FIELDS);
   @@ -370,6 +392,12 @@ public class TestShreddedObject {
        return Variants.value(metadata, slice);
      }
    
   +  private static ShreddedObject createUnserializedObject(Map<String, 
VariantValue> fields) {
   +    ByteBuffer metadataBuffer = 
VariantTestUtil.createMetadata(fields.keySet(), false);
   +    VariantMetadata metadata = SerializedMetadata.from(metadataBuffer);
   +    return new ShreddedObject(metadata, createShreddedObject(metadata, 
fields));
   +  }
   +
      /** Creates a ShreddedObject with fields in its shredded map, using the 
given metadata */
      private static ShreddedObject createShreddedObject(
          VariantMetadata metadata, Map<String, VariantValue> fields) {
   ```
   
   Feel free to use that test case. It would also be nice to have a Parquet 
test like this one. If you end up keeping it, please use helper methods to 
create the Parquet schema (see `ParquetVariantUtil`).



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to