dirtysalt opened a new issue, #15086:
URL: https://github.com/apache/iceberg/issues/15086

   ## Apache Iceberg version
   1.10.1 (latest release)
   Also tested: 1.10.0
   
   ## Query engine
   Other (Java API)
   
   ## Please describe the bug
   
   ### Summary
   When using `variantShreddingFunc` to partially shred variant fields (i.e., 
shredding only a subset of fields), unshredded fields are completely lost 
during the write operation. The binary `value` field contains only metadata 
headers and zeros instead of the actual field values, causing permanent data 
loss.
   
   ### Expected Behavior
   When a variant has fields {id, name, city} and only `id` is shredded into 
`typed_value`:
   - `id` should be stored in the `typed_value` group structure
   - `name` and `city` should be serialized and stored in the binary `value` 
field
   - On read, the variant should be reconstructed with all three fields intact
   
   ### Actual Behavior
   - Schema generation works correctly: only `id` appears in `typed_value` 
structure
   - Binary `value` field contains only placeholder bytes (zeros), not the 
actual unshredded field values
   - Reading fails with `IndexOutOfBoundsException` in 
`SerializedObject.initOffsetsAndLengths()`
   - Unshredded field data is permanently lost
   
   ### Impact
   This is a **critical data loss bug** that makes partial variant shredding 
completely unusable. Full shredding (all fields) and no shredding work 
correctly, but partial shredding loses data.
   
   ### Reproduction Steps
   
   **Test Environment:**
   - Iceberg version: 1.10.1 (also reproduced on 1.10.0)
   - Maven dependency:
     \`\`\`xml
     <dependency>
         <groupId>org.apache.iceberg</groupId>
         <artifactId>iceberg-parquet</artifactId>
         <version>1.10.1</version>
     </dependency>
     \`\`\`
   
   **Minimal Reproduction Code:**
   
   \`\`\`java
   package com.starrocks.lab;
   
   import org.apache.hadoop.conf.Configuration;
   import org.apache.iceberg.data.GenericRecord;
   import org.apache.iceberg.data.Record;
   import org.apache.iceberg.data.parquet.GenericParquetWriter;
   import org.apache.iceberg.io.FileAppender;
   import org.apache.iceberg.io.OutputFile;
   import org.apache.iceberg.parquet.Parquet;
   import org.apache.iceberg.Schema;
   import org.apache.iceberg.types.Types;
   import org.apache.iceberg.variants.ShreddedObject;
   import org.apache.iceberg.variants.Variant;
   import org.apache.iceberg.variants.VariantMetadata;
   import org.apache.iceberg.variants.Variants;
   import org.apache.parquet.hadoop.ParquetFileReader;
   import org.apache.parquet.hadoop.metadata.ParquetMetadata;
   import org.apache.parquet.column.page.PageReadStore;
   import org.apache.parquet.example.data.Group;
   import org.apache.parquet.example.data.simple.convert.GroupRecordConverter;
   import org.apache.parquet.io.ColumnIOFactory;
   import org.apache.parquet.io.MessageColumnIO;
   import org.apache.parquet.io.RecordReader;
   import org.apache.parquet.schema.MessageType;
   import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
   import org.apache.parquet.schema.Type;
   
   import java.io.File;
   import java.io.IOException;
   import java.util.ArrayList;
   import java.util.List;
   
   public class MinimalVariantShreddingBugTest {
       
       private static final Schema SCHEMA = new Schema(
               Types.NestedField.optional(1, "data", Types.VariantType.get())
       );
       
       public static void main(String[] args) throws IOException {
           String outputPath = "/tmp/variant-bug-test.parquet";
           File outputFile = new File(outputPath);
           if (outputFile.exists()) {
               outputFile.delete();
           }
           
           // Create variants with 3 fields: id, name, city
           // Shred only 'id', leaving 'name' and 'city' unshredded
           List<Record> records = buildTestRecords();
           
           // Write with partial shredding
           OutputFile out = org.apache.iceberg.Files.localOutput(outputFile);
           Parquet.WriteBuilder builder = Parquet.write(out)
                   .schema(SCHEMA)
                   .createWriterFunc(GenericParquetWriter::create)
                   
.variantShreddingFunc(MinimalVariantShreddingBugTest::shreddedType);
           
           try (FileAppender<Record> appender = builder.build()) {
               for (Record r : records) {
                   appender.add(r);
               }
           }
           
           System.out.println("Written " + records.size() + " records");
           
           // Verify the binary value field
           verifyParquetFile(outputFile);
       }
       
       /**
        * Shredding function: only 'id' field goes to typed_value.
        * 'name' and 'city' should remain in binary value.
        */
       private static Type shreddedType(int fieldId, String name) {
           return org.apache.parquet.schema.Types.optionalGroup()
                   .id(fieldId)
                   .addField(shreddedField("id",
                           
org.apache.parquet.schema.Types.optional(PrimitiveTypeName.BINARY).named("value"),
                           
org.apache.parquet.schema.Types.optional(PrimitiveTypeName.INT64).named("typed_value")))
                   .named("typed_value");
       }
       
       private static Type shreddedField(String fieldName, Type valueField, 
Type typedValueField) {
           return org.apache.parquet.schema.Types.optionalGroup()
                   .addField(valueField)
                   .addField(typedValueField)
                   .named(fieldName);
       }
       
       private static List<Record> buildTestRecords() {
           List<Record> records = new ArrayList<>();
           VariantMetadata metadata = Variants.metadata("id", "name", "city");
           
           for (int i = 0; i < 3; i++) {
               Record rec = GenericRecord.create(SCHEMA.asStruct());
               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 value = Variant.of(metadata, obj);
               rec.setField("data", value);
               records.add(rec);
           }
           
           return records;
       }
       
       private static void verifyParquetFile(File parquetFile) throws 
IOException {
           try (ParquetFileReader reader = ParquetFileReader.open(
                   new Configuration(), new 
org.apache.hadoop.fs.Path(parquetFile.getAbsolutePath()))) {
               
               ParquetMetadata metadata = reader.getFooter();
               MessageType schema = metadata.getFileMetaData().getSchema();
               
               PageReadStore pages = reader.readNextRowGroup();
               if (pages != null) {
                   MessageColumnIO columnIO = new 
ColumnIOFactory().getColumnIO(schema);
                   RecordReader<Group> recordReader = columnIO.getRecordReader(
                           pages, new GroupRecordConverter(schema));
                   
                   for (int i = 0; i < pages.getRowCount(); i++) {
                       Group record = recordReader.read();
                       Group dataGroup = record.getGroup("data", 0);
                       
                       if (dataGroup.getFieldRepetitionCount("value") > 0) {
                           byte[] valueBytes = dataGroup.getBinary("value", 
0).getBytes();
                           
                           System.out.println("Row " + i + " binary value 
(hex): " + bytesToHex(valueBytes));
                           
                           boolean hasName = containsString(valueBytes, "user_" 
+ i);
                           boolean hasCity = containsString(valueBytes, "city_" 
+ i);
                           
                           System.out.println("  Contains 'user_" + i + "': " + 
hasName);
                           System.out.println("  Contains 'city_" + i + "': " + 
hasCity);
                           
                           if (!hasName || !hasCity) {
                               System.out.println("  BUG: Unshredded fields 
missing from binary value!");
                           }
                       }
                   }
               }
           }
       }
       
       private static String bytesToHex(byte[] bytes) {
           StringBuilder sb = new StringBuilder();
           for (byte b : bytes) {
               sb.append(String.format("%02x", b));
           }
           return sb.toString();
       }
       
       private static boolean containsString(byte[] bytes, String search) {
           byte[] searchBytes = 
search.getBytes(java.nio.charset.StandardCharsets.UTF_8);
           for (int i = 0; i <= bytes.length - searchBytes.length; i++) {
               boolean found = true;
               for (int j = 0; j < searchBytes.length; j++) {
                   if (bytes[i + j] != searchBytes[j]) {
                       found = false;
                       break;
                   }
               }
               if (found) {
                   return true;
               }
           }
           return false;
       }
   }
   \`\`\`
   
   **Expected Output:**
   \`\`\`
   Row 0 binary value (hex): [should contain UTF-8 encoded "user_0" and 
"city_0"]
     Contains 'user_0': true
     Contains 'city_0': true
   \`\`\`
   
   **Actual Output:**
   \`\`\`
   Row 0 binary value (hex): 020200000000000000000000000000000000000000
     Contains 'user_0': false
     Contains 'city_0': false
     BUG: Unshredded fields missing from binary value!
   \`\`\`
   
   ### Technical Analysis
   
   **Binary Value Field Structure:**
   \`\`\`
   Byte 0: 02 = header byte
   Byte 1: 02 = num_fields (indicates 2 unshredded fields)
   Remaining: 00 00 00... = all zeros (no actual field data)
   \`\`\`
   
   The header correctly indicates 2 unshredded fields exist, but the 
serialization logic fails to write their actual values.
   
   **Root Cause:**
   The variant writer correctly excludes unshredded fields from the 
\`typed_value\` structure but fails to serialize them into the binary \`value\` 
field. The \`ShreddedObject\` serialization logic appears to only handle fields 
defined in the shredding schema, ignoring the remaining fields entirely.
   
   **Affected Components:**
   - \`org.apache.iceberg.variants.ShreddedObject\` serialization
   - \`org.apache.iceberg.parquet.ParquetVariantWriters\` write path
   - Variant value field encoding logic
   
   ### Workarounds
   
   Until this is fixed, avoid partial shredding:
   1. Either shred all fields (full shredding)
   2. Or shred no fields (no shredding)
   
   Both approaches work correctly. Only partial shredding exhibits this data 
loss bug.
   
   ### Stack Trace (on read attempt)
   
   \`\`\`
   java.lang.IndexOutOfBoundsException: Index 0 out of bounds for length 0
       at 
java.base/jdk.internal.util.Preconditions.outOfBounds(Preconditions.java:64)
       at 
java.base/jdk.internal.util.Preconditions.outOfBoundsCheckIndex(Preconditions.java:70)
       at 
java.base/jdk.internal.util.Preconditions.checkIndex(Preconditions.java:266)
       at java.base/java.util.Objects.checkIndex(Objects.java:359)
       at 
org.apache.iceberg.variants.SerializedObject.initOffsetsAndLengths(SerializedObject.java:122)
       at 
org.apache.iceberg.variants.SerializedObject.<init>(SerializedObject.java:54)
   \`\`\`
   
   ### Additional Context
   
   Complete test suite available at:
   
https://github.com/dirtysalt/codes/tree/master/java/IcebergVariantShreddingTest
   
   The issue has been verified with:
   - Direct Parquet binary inspection
   - Multiple test scenarios
   - Both Iceberg 1.10.0 and 1.10.1
   
   ## Willingness to contribute
   
   I would be willing to contribute a fix for this bug with guidance from the 
Iceberg community.


-- 
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