@@ -69,6 +69,11 @@ public RecordBinder(org.apache.iceberg.Schema icebergSchema, Schema avroSchema,
69
69
this .nestedStructBinders = precomputeNestedStructBinders (typeAdapter );
70
70
}
71
71
72
+
73
+ public org .apache .iceberg .Schema getIcebergSchema () {
74
+ return icebergSchema ;
75
+ }
76
+
72
77
/**
73
78
* Creates a new immutable Record view of the given Avro record.
74
79
* Each call returns a separate instance with its own data reference.
@@ -82,13 +87,23 @@ public Record bind(GenericRecord avroRecord) {
82
87
}
83
88
84
89
private void initializeFieldMappings (Schema avroSchema ) {
90
+ Schema recordSchema = avroSchema ;
91
+
92
+ if (recordSchema .getType () == Schema .Type .UNION ) {
93
+ recordSchema = recordSchema .getTypes ().stream ()
94
+ .filter (s -> s .getType () == Schema .Type .RECORD )
95
+ .findFirst ()
96
+ .orElseThrow (() -> new IllegalArgumentException ("UNION schema does not contain a RECORD type: " + avroSchema ));
97
+ }
98
+
85
99
for (int icebergPos = 0 ; icebergPos < icebergSchema .columns ().size (); icebergPos ++) {
86
100
Types .NestedField icebergField = icebergSchema .columns ().get (icebergPos );
87
101
String fieldName = icebergField .name ();
88
102
89
- Schema .Field avroField = avroSchema .getField (fieldName );
103
+ Schema .Field avroField = recordSchema .getField (fieldName );
90
104
if (avroField != null ) {
91
105
fieldMappings [icebergPos ] = createOptimizedMapping (
106
+ avroField .name (),
92
107
avroField .pos (),
93
108
icebergField .type (),
94
109
avroField .schema ()
@@ -99,19 +114,14 @@ private void initializeFieldMappings(Schema avroSchema) {
99
114
}
100
115
}
101
116
102
- private FieldMapping createOptimizedMapping (int avroPosition , Type icebergType , Schema avroType ) {
103
- FieldMapping mapping = new FieldMapping ();
104
- mapping .avroPosition = avroPosition ;
105
- mapping .icebergType = icebergType ;
106
- mapping .typeId = icebergType .typeId ();
107
- mapping .avroSchema = avroType ;
108
-
117
+ private FieldMapping createOptimizedMapping (String avroFieldName , int avroPosition , Type icebergType , Schema avroType ) {
118
+ org .apache .iceberg .Schema nestedSchema = null ;
119
+ String nestedSchemaId = null ;
109
120
if (icebergType .isStructType ()) {
110
- mapping . nestedSchema = icebergType .asStructType ().asSchema ();
111
- mapping . nestedSchemaId = icebergType .toString ();
121
+ nestedSchema = icebergType .asStructType ().asSchema ();
122
+ nestedSchemaId = icebergType .toString ();
112
123
}
113
-
114
- return mapping ;
124
+ return new FieldMapping (avroPosition , avroFieldName , icebergType , icebergType .typeId (), avroType , nestedSchema , nestedSchemaId );
115
125
}
116
126
117
127
@@ -122,12 +132,12 @@ private Map<String, RecordBinder> precomputeNestedStructBinders(TypeAdapter<Sche
122
132
Map <String , RecordBinder > binders = new HashMap <>();
123
133
124
134
for (FieldMapping mapping : fieldMappings ) {
125
- if (mapping != null && mapping .typeId == Type .TypeID .STRUCT ) {
126
- String structId = mapping .nestedSchemaId ;
135
+ if (mapping != null && mapping .typeId () == Type .TypeID .STRUCT ) {
136
+ String structId = mapping .nestedSchemaId () ;
127
137
if (!binders .containsKey (structId )) {
128
138
RecordBinder nestedBinder = new RecordBinder (
129
- mapping .nestedSchema ,
130
- mapping .avroSchema ,
139
+ mapping .nestedSchema () ,
140
+ mapping .avroSchema () ,
131
141
typeAdapter
132
142
);
133
143
binders .put (structId , nestedBinder );
@@ -170,18 +180,18 @@ public Object get(int pos) {
170
180
}
171
181
172
182
FieldMapping mapping = fieldMappings [pos ];
173
- if (mapping == null ) {
183
+ if (mapping == null || ! avroRecord . hasField ( mapping . avroKey ()) ) {
174
184
return null ;
175
185
}
176
186
177
- Object avroValue = avroRecord .get (mapping .avroPosition );
187
+ Object avroValue = avroRecord .get (mapping .avroPosition () );
178
188
if (avroValue == null ) {
179
189
return null ;
180
190
}
181
191
182
192
// Handle STRUCT type
183
- if (mapping .typeId == Type .TypeID .STRUCT ) {
184
- String structId = mapping .nestedSchemaId ;
193
+ if (mapping .typeId () == Type .TypeID .STRUCT ) {
194
+ String structId = mapping .nestedSchemaId () ;
185
195
RecordBinder nestedBinder = nestedStructBinders .get (structId );
186
196
if (nestedBinder == null ) {
187
197
throw new IllegalStateException ("Nested binder not found for struct: " + structId );
@@ -190,7 +200,7 @@ public Object get(int pos) {
190
200
}
191
201
192
202
// Delegate conversion of all other types to the adapter
193
- return typeAdapter .convert (avroValue , mapping .avroSchema , mapping .icebergType );
203
+ return typeAdapter .convert (avroValue , mapping .avroSchema () , mapping .icebergType () );
194
204
}
195
205
196
206
@ Override
@@ -235,13 +245,50 @@ public <T> void set(int pos, T value) {
235
245
236
246
// Field mapping structure
237
247
private static class FieldMapping {
238
- int avroPosition ;
239
- Type icebergType ;
240
- Type .TypeID typeId ;
241
- Schema avroSchema ;
242
-
243
- // For struct types
244
- org .apache .iceberg .Schema nestedSchema ;
245
- String nestedSchemaId ;
248
+ private final int avroPosition ;
249
+ private final String avroKey ;
250
+ private final Type icebergType ;
251
+ private final Type .TypeID typeId ;
252
+ private final Schema avroSchema ;
253
+ private final org .apache .iceberg .Schema nestedSchema ;
254
+ private final String nestedSchemaId ;
255
+
256
+ FieldMapping (int avroPosition , String avroKey , Type icebergType , Type .TypeID typeId , Schema avroSchema , org .apache .iceberg .Schema nestedSchema , String nestedSchemaId ) {
257
+ this .avroPosition = avroPosition ;
258
+ this .avroKey = avroKey ;
259
+ this .icebergType = icebergType ;
260
+ this .typeId = typeId ;
261
+ this .avroSchema = avroSchema ;
262
+ this .nestedSchema = nestedSchema ;
263
+ this .nestedSchemaId = nestedSchemaId ;
264
+ }
265
+
266
+ public int avroPosition () {
267
+ return avroPosition ;
268
+ }
269
+
270
+ public String avroKey () {
271
+ return avroKey ;
272
+ }
273
+
274
+ public Type icebergType () {
275
+ return icebergType ;
276
+ }
277
+
278
+ public Type .TypeID typeId () {
279
+ return typeId ;
280
+ }
281
+
282
+ public Schema avroSchema () {
283
+ return avroSchema ;
284
+ }
285
+
286
+ public org .apache .iceberg .Schema nestedSchema () {
287
+ return nestedSchema ;
288
+ }
289
+
290
+ public String nestedSchemaId () {
291
+ return nestedSchemaId ;
292
+ }
246
293
}
247
294
}
0 commit comments