@@ -157,6 +157,27 @@ private static class AvroPartnerAccessor implements PartnerAccessor<Schema, Sche
157157
158158 private static final Schema MAP_KEY = Schema .create (Schema .Type .STRING );
159159
160+ /**
161+ * Extracts the actual type from a schema that may be a union.
162+ * Handles two cases:
163+ * 1. Single-element unions like [{"type":"record",...}] - extracts the single type
164+ * 2. Nullable unions like ["null", "string"] - extracts the non-null type
165+ *
166+ * @param schema The schema to extract from
167+ * @return The extracted schema
168+ */
169+ private static Schema extractFromUnion (Schema schema ) {
170+ if (schema .getType () == Schema .Type .UNION ) {
171+ List <Schema > types = schema .getTypes ();
172+ if (types .size () == 1 ) {
173+ // Single-element union: just extract that single type
174+ return types .get (0 );
175+ }
176+ }
177+ // Extract the non-null type from nullable unions like ["null", "string"]
178+ return SchemaUtilities .extractIfOption (schema );
179+ }
180+
160181 @ Override
161182 public Schema .Field fieldPartner (Schema partner , String fieldName ) {
162183 Schema schema = SchemaUtilities .extractIfOption (partner );
@@ -180,18 +201,7 @@ public Schema mapValuePartner(Schema partner) {
180201 if (schema .getType () != Schema .Type .MAP ) {
181202 return null ;
182203 }
183- Schema valueType = schema .getValueType ();
184- // Handle single-element unions like [{"type":"record",...}]
185- // These are not typical nullable unions but still valid Avro unions
186- if (valueType .getType () == Schema .Type .UNION ) {
187- List <Schema > types = valueType .getTypes ();
188- if (types .size () == 1 ) {
189- // Single-element union: just extract that single type
190- return types .get (0 );
191- }
192- }
193- // Extract the non-null type from nullable unions like ["null", "string"]
194- return SchemaUtilities .extractIfOption (valueType );
204+ return extractFromUnion (schema .getValueType ());
195205 }
196206
197207 @ Override
@@ -200,18 +210,7 @@ public Schema listElementPartner(Schema partner) {
200210 if (schema .getType () != Schema .Type .ARRAY ) {
201211 return null ;
202212 }
203- Schema elementType = schema .getElementType ();
204- // Handle single-element unions like [{"type":"record",...}]
205- // These are not typical nullable unions but still valid Avro unions
206- if (elementType .getType () == Schema .Type .UNION ) {
207- List <Schema > types = elementType .getTypes ();
208- if (types .size () == 1 ) {
209- // Single-element union: just extract that single type
210- return types .get (0 );
211- }
212- }
213- // Extract the non-null type from nullable unions like ["null", "string"]
214- return SchemaUtilities .extractIfOption (elementType );
213+ return extractFromUnion (schema .getElementType ());
215214 }
216215
217216 @ Override
0 commit comments