dirtysalt commented on code in PR #15087:
URL: https://github.com/apache/iceberg/pull/15087#discussion_r2714996457
##########
parquet/src/test/java/org/apache/iceberg/parquet/TestVariantWriters.java:
##########
@@ -280,4 +282,53 @@ 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
Review Comment:
@RussellSpitzer I may not have enough knowledge about iceberg code, and
pardon me if I'm wrong because I just do some tests and observe results, and
try to connect them and find reasonable explanation. Maybe my explanation is
not the root cause.
I think it's relevant to the fix. The bug is not "we lose the unshredded
fields", but is "we will lose shredded fields if we partially shred some
fields". If I write the test case like following "shred all fields", then
the test case works.
```diff
VariantShreddingFunction partialShredding =
(id, name) -> {
- VariantMetadata shreddedMetadata = Variants.metadata("id");
- ShreddedObject shreddedObject = Variants.object(shreddedMetadata);
- shreddedObject.put("id", Variants.of(1234L));
- return ParquetVariantUtil.toParquetSchema(shreddedObject);
+ if (name.equals("var")) {
+ ShreddedObject obj = Variants.object(metadata);
+ obj.put("id", Variants.of(1000L));
+ obj.put("name", Variants.of("user"));
+ obj.put("city", Variants.of("city"));
+ return ParquetVariantUtil.toParquetSchema(obj);
+ }
+ return null;
};
```
But if I just do partially shredding, then exception happens.
----------
Actually, the fix should be following, only to shred the `id` field in
variant. But I guess it does not matter much because `id` field of primitive
type does not support shredding.
```diff
---
a/parquet/src/test/java/org/apache/iceberg/parquet/TestVariantWriters.java
+++
b/parquet/src/test/java/org/apache/iceberg/parquet/TestVariantWriters.java
@@ -302,13 +302,16 @@ public class TestVariantWriters {
records.add(record);
}
- // Shredding function that only shreds the "id" field
+ // Shredding function that only shreds the "id" field in the variant
VariantShreddingFunction partialShredding =
(id, name) -> {
- VariantMetadata shreddedMetadata = Variants.metadata("id");
- ShreddedObject shreddedObject = Variants.object(shreddedMetadata);
- shreddedObject.put("id", Variants.of(1234L));
- return ParquetVariantUtil.toParquetSchema(shreddedObject);
+ if (name.equals("var")) {
+ VariantMetadata shreddedMetadata = Variants.metadata("id");
+ ShreddedObject shreddedObject =
Variants.object(shreddedMetadata);
+ shreddedObject.put("id", Variants.of(1234L));
+ return ParquetVariantUtil.toParquetSchema(shreddedObject);
+ }
+ return null;
};
```
--
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]