fix(manifest): encode day-transform partition fields with Avro date logical type#915
fix(manifest): encode day-transform partition fields with Avro date logical type#915artbcf wants to merge 1 commit into
Conversation
laskoviymishka
left a comment
There was a problem hiding this comment.
Fix works, it's safe, and it unblocks Trino/Spark readers, i think it's fine to merge as-is.
That said, the real divergence is in DayTransform.ResultType returning Int32 instead of DateType (Java and PyIceberg both return Date). This PR patches the symptom in partitionTypeToAvroSchema, but the same ResultType divergence also leaves table/internal/utils.go building a wrong fieldIDToLogicalType for day partitions — masked today only because ManifestWriter.addEntry overrides it. Any consumer using DataFileBuilder directly sees the mismatch.
So: fine to land as-is, but worth a follow-up issue (or a second PR) for the root fix — either at ResultType or inside PartitionSpec.PartitionType(). That also drops the new spec parameter here and clears utils.go in one move.
The convertDateValue no-op branch and the empty-spec tests are worth tightening regardless.
| // partitionTypeToAvroSchema converts the partition struct type to an Avro | ||
| // schema. spec is used to resolve the transform for each partition field so | ||
| // that Int32 fields produced by DayTransform are encoded with the "date" | ||
| // logical type, matching what other engines (Trino, Spark, etc.) write. | ||
| func partitionTypeToAvroSchema(t *StructType, spec PartitionSpec) (*avro.Schema, error) { | ||
| // Build a field-ID → Transform map so we can detect DayTransform below. | ||
| transformByFieldID := make(map[int]Transform, spec.NumFields()) | ||
| for i := range spec.NumFields() { | ||
| pf := spec.Field(i) | ||
| transformByFieldID[pf.FieldID] = pf.Transform | ||
| } | ||
|
|
||
| fields := make([]avro.SchemaField, len(t.FieldList)) | ||
| for i, f := range t.FieldList { | ||
| var node avro.SchemaNode | ||
| switch typ := f.Type.(type) { | ||
| case Int32Type: |
There was a problem hiding this comment.
partitionTypeToAvroSchema(t StructType, spec PartitionSpec) was a pure type-to-schema mapping. It now takes the partition spec and builds a transform lookup to decide one field's Avro type. That coupling exists only because Int32Type is hiding what should be a DateType (see context above).
A few alternatives, roughly from cleanest to most surgical:
-
Fix
DayTransform.ResultTypeto returnDateType. Matches Java and PyIceberg.PartitionSpec.PartitionType()then produces aDateTypefield for day partitions, the existingcase DateType:branch fires, the new parameter goes away, andtable/internal/utils.gofixes itself. Wider blast (anyone readingResultTypeorBoundTransform.Type()seesDatenow), but that's the correct answer per spec. -
Fix inside
PartitionSpec.PartitionType(schema). Same effect on the Avro mapping without touchingResultType: when building the partition struct, override the field type toDateTypefor day transforms.partitionTypeToAvroSchemastays pure,utils.gosees the right type, blast radius stays narrow to the partition-struct construction path. -
Narrow the parameter. If option 1 or 2 isn't on the table, pass
map[int]Transform(or afunc(fieldID int) Transformclosure) instead of the fullPartitionSpec. Keeps the coupling minimal and makes the dependency explicit at the call site.
Option 2 is probably the best trade-off — fixes the root symptom without the wider implications of changing ResultType.
What's you take on this? I don't have strong opinion, but current approach seems to me a bit too hacky.
|
|
||
| got, ok := decoded["bucket_id_identity"] | ||
| require.True(t, ok) | ||
| assert.IsType(t, int32(0), got, "plain Int32 partition must decode as int32, not time.Time") |
There was a problem hiding this comment.
"date" is a common substring: a field named update_date or dated_id would make this pass accidentally. The assert.IsType(time.Time{}, got) check a few lines above is a stronger signal (it only decodes as time.Time when the logical type is applied).
Drop the Contains/NotContains and rely on the decode-type assertion?
Or marshal the schema to JSON and assert on the structured logicalType field?
…ogical type DayTransform.Apply() returns int32 (days since epoch) but the Avro encoding of a day-partition column must carry the 'date' logical type so that Trino, Spark, and other Iceberg engines can read manifests. Root fix: PartitionSpec.PartitionType() now overrides DayTransform's ResultType (Int32) to DateType for the partition struct it produces. The existing DateType branch in partitionTypeToAvroSchema then emits DateNode (int + date logical type) automatically — no coupling between the Avro layer and transform internals. This also fixes DataFileStatistics.ToDataFile() in table/internal/utils.go, which builds fieldIDToLogicalType by switching on ResultType: the case DateType branch now fires correctly for day-partition fields. Tests verify that day-partition fields decode as time.Time (confirming the date logical type is applied) while plain Int32 fields decode as int32.
bf8e4d0 to
ac8bc4a
Compare
The actual spec says that the result type should be int, not date. If both iceberg-java and pyiceberg are using date, and the existing engines expect it to be a date, then we should probably have a discussion about the spec. @Fokko @kevinjqliu would either of you be up for commenting on whether or not this is something I should bring up on the mailing list? I'd prefer to get clarity here before having iceberg-go implement something that's contrary to the spec. |
|
ok i had a few paragraphs i was going to write here but ended up deleting it because i think this warrants a bigger discussion 😄 i think every implementation has been tripped up by this haha java
python
rust and likely many more threads. |
|
Ok i created an umbrella issue here apache/iceberg#16414 (and devlist thread https://lists.apache.org/thread/qqw5oog5swmswxqqmp693vz1rw132xb6) For iceberg-go, i would suggest to align with the other implementation:
This follows Postel’s Law, "Be liberal in what you accept, and conservative in what you send." |
btw based on my research, it seems that Java should be able to read the Avro int. Curious what was the error you saw. |
| continue | ||
| } | ||
| resultType := field.Transform.ResultType(sourceType) | ||
| // DayTransform.ResultType returns Int32 (days since epoch), but the |
There was a problem hiding this comment.
i would suggest changing the DayTransform's ResultType to PrimitiveTypes.Date instead
Line 792 in 1a44611
so iceberg-go is better aligned with other implementations (see apache/iceberg#16414)
There was a problem hiding this comment.
agreed, let's get that updated first before we merge this!
I can confirm, Trino can read both "int" and "int + date logical type" The issue was on the ingestion side. A compaction (based on iceberg-rust) normalized everything to "int + date logical type", which caused iceberg-go to refuse to append new files with the partition column encoded as "int". |
thanks for the additional context! This makes sense. All the other implementations can accept Avro int and Avro date (per apache/iceberg#16414) Orthogonally, Iceberg-go should change to write as Avro date instead of Avro int, to align with the other implementations. |
|
okay cool, thanks @kevinjqliu It's funny because we originally did use the |
| continue | ||
| } | ||
| resultType := field.Transform.ResultType(sourceType) | ||
| // DayTransform.ResultType returns Int32 (days since epoch), but the |
There was a problem hiding this comment.
agreed, let's get that updated first before we merge this!
DayTransform.Apply() returns an int32 (days since epoch) but partitionTypeToAvroSchema was encoding all Int32 partition fields as plain Avro int, without the 'date' logical type. This caused manifest files to be unreadable by Trino, Spark, and other Iceberg engines that require the date logical type on day-partition columns.
Two fixes:
Tests added for both the schema encoding and the plain-int case.