dirtysalt opened a new issue, #15085:
URL: https://github.com/apache/iceberg/issues/15085
### Apache Iceberg version
1.10.0, 1.10.1
### Query engine
None (Java API, Parquet write directly)
### Please describe the bug 🐞
When using `variantShreddingFunc` to partially shred variant fields
(shredding only **some** fields, not all), the unshredded fields are
**completely lost**. Their values are not written to the binary `value` field,
causing **data loss**.
**Impact**: Critical - partial shredding is completely unusable due to data
loss.
#### Expected Behavior
When using a shredding function that excludes certain fields from
`typed_value`:
1. ✅ Excluded fields should NOT appear in the Parquet `typed_value` group
(works)
2. ❌ Excluded fields should be stored in the binary `value` field (BROKEN)
3. ❌ Reading should reconstruct the full variant with both shredded and
unshredded fields (BROKEN - `IndexOutOfBoundsException`)
#### Actual Behavior
1. ✅ Parquet schema correctly excludes unshredded fields from `typed_value`
2. ❌ Binary `value` field contains only field ID placeholders (zeros), **no
actual data**
3. ❌ Reading fails with `java.lang.IndexOutOfBoundsException` in
`SerializedObject.initOffsetsAndLengths`
### Minimal Reproduction
```java
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.schema.PrimitiveType.PrimitiveTypeName;
import org.apache.parquet.schema.Type;
import java.io.File;
import java.util.ArrayList;
import java.util.List;
public class VariantShreddingBugTest {
private static final Schema SCHEMA = new Schema(
Types.NestedField.optional(1, "data", Types.VariantType.get())
);
public static void main(String[] args) throws Exception {
// Create records with 3 fields: id, name, city
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)); // Should be in
binary value
obj.put("city", Variants.of("city_" + i)); // Should be in
binary value
Variant value = Variant.of(metadata, obj);
rec.setField("data", value);
records.add(rec);
}
// Write with shredding: only 'id' goes to typed_value
// 'name' and 'city' should stay in binary value
File outputFile = new File("/tmp/variant-bug.parquet");
OutputFile out = org.apache.iceberg.Files.localOutput(outputFile);
Parquet.WriteBuilder builder = Parquet.write(out)
.schema(SCHEMA)
.createWriterFunc(GenericParquetWriter::create)
.variantShreddingFunc((fieldId, name) ->
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")))
// 'name' and 'city' intentionally omitted from
typed_value
.named("typed_value")
);
try (FileAppender<Record> appender = builder.build()) {
for (Record r : records) {
appender.add(r);
}
}
System.out.println("File written: " + outputFile.getAbsolutePath());
// Inspect the file - binary 'value' field will only contain zeros
}
private static Type shreddedField(String fieldName, Type valueField,
Type typedValueField) {
return org.apache.parquet.schema.Types.optionalGroup()
.addField(valueField)
.addField(typedValueField)
.named(fieldName);
}
}
```
#### Maven Dependencies
```xml
<properties>
<iceberg.version>1.10.1</iceberg.version>
</properties>
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-core</artifactId>
<version>${iceberg.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-parquet</artifactId>
<version>${iceberg.version}</version>
</dependency>
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-data</artifactId>
<version>${iceberg.version}</version>
</dependency>
```
### Verification of Bug
Run the above code and inspect the output Parquet file:
**Parquet Schema (correct):**
```
optional group data (VARIANT(1)) {
required binary metadata;
optional binary value;
optional group typed_value {
optional group id {
optional binary value;
optional int64 typed_value;
}
// ✅ 'name' and 'city' correctly excluded from typed_value
}
}
```
**Binary value field (incorrect - DATA LOSS):**
Using ParquetFileReader to inspect the binary `value` field:
```
Row 0: 020200000000000000000000000000000000000000
Row 1: 020200000000000000000000000000000000000000
Row 2: 020200000000000000000000000000000000000000
```
The binary value contains:
- Byte 0: `0x02` (header)
- Byte 1: `0x02` (num_fields = 2, for name and city)
- **Rest: All zeros (field IDs but NO actual string data)**
**❌ Expected:** Binary value should contain UTF-8 encoded strings "user_0",
"city_0", etc.
**❌ Actual:** Strings are completely missing - **DATA LOSS**
### Root Cause
The variant serialization logic in Iceberg 1.10.x appears to:
1. ✅ Correctly build the Parquet schema structure for partial shredding
2. ❌ Only serialize fields that are in the shredding definition to
`typed_value`
3. ❌ **Fail to serialize unshredded fields to the binary `value` field**
4. ❌ Write only structural placeholders (field IDs) without actual data
### Read Failure
Attempting to read the shredded file also fails:
```
java.lang.IndexOutOfBoundsException: Index 1 out of bounds for length 1
at java.base/java.util.ArrayList.get(ArrayList.java:427)
at
org.apache.iceberg.variants.SerializedObject.initOffsetsAndLengths(SerializedObject.java:105)
at
org.apache.iceberg.variants.SerializedObject.<init>(SerializedObject.java:80)
at
org.apache.iceberg.variants.SerializedObject.from(SerializedObject.java:49)
```
### Workarounds
Currently unusable. To work around:
1. **Shred ALL fields** (include every field in `typed_value`), or
2. **Use no shredding** (don't use `variantShreddingFunc` at all)
**Partial shredding cannot be used.**
### Environment
- **Iceberg Version**: 1.10.0, 1.10.1
- **Java Version**: 17
- **Parquet Version**: 1.16.0 (bundled with Iceberg)
- **Hadoop Version**: 3.4.1
### Additional Context
This bug affects the core value proposition of variant shredding - the
ability to selectively shred frequently-queried fields while keeping other
fields in binary format. Without partial shredding support, users must either:
- Shred everything (losing flexibility)
- Shred nothing (losing performance benefits)
### Willingness to contribute
- [ ] I can contribute a fix for this bug independently
- [x] I would be willing to contribute a fix for this bug with guidance from
the Iceberg community
- [ ] I cannot contribute a fix for this bug at this time
I can provide additional test cases, verification scripts, or debugging
information as needed.
--
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]