31
31
import org .apache .avro .Schema ;
32
32
33
33
import java .util .ArrayList ;
34
+ import java .util .Collections ;
35
+ import java .util .Comparator ;
34
36
import java .util .Deque ;
35
37
import java .util .HashMap ;
36
38
import java .util .Iterator ;
37
39
import java .util .LinkedList ;
38
40
import java .util .List ;
39
41
import java .util .Map ;
40
42
import java .util .concurrent .atomic .AtomicInteger ;
43
+ import java .util .stream .Collectors ;
41
44
42
45
import static org .apache .avro .Schema .Type .UNION ;
43
46
@@ -117,10 +120,18 @@ public static Schema convert(Type type, String name) {
117
120
118
121
/** Convert an avro schema into internal type. */
119
122
public static Type convertToField (Schema schema ) {
120
- return buildTypeFromAvroSchema (schema );
123
+ return buildTypeFromAvroSchema (schema , Collections .emptyMap ());
124
+ }
125
+
126
+ private static Type convertToField (Schema schema , Map <String , Integer > existingFieldNameToPositionMapping ) {
127
+ return buildTypeFromAvroSchema (schema , existingFieldNameToPositionMapping );
121
128
}
122
129
123
130
/** Convert an avro schema into internalSchema. */
131
+ public static InternalSchema convert (Schema schema , Map <String , Integer > existingFieldNameToPositionMapping ) {
132
+ return new InternalSchema ((Types .RecordType ) convertToField (schema , existingFieldNameToPositionMapping ));
133
+ }
134
+
124
135
public static InternalSchema convert (Schema schema ) {
125
136
return new InternalSchema ((Types .RecordType ) convertToField (schema ));
126
137
}
@@ -151,11 +162,11 @@ public static Schema nullableSchema(Schema schema) {
151
162
* @param schema a avro schema.
152
163
* @return a hudi type.
153
164
*/
154
- public static Type buildTypeFromAvroSchema (Schema schema ) {
165
+ public static Type buildTypeFromAvroSchema (Schema schema , Map < String , Integer > existingNameToPositions ) {
155
166
// set flag to check this has not been visited.
156
- Deque <String > visited = new LinkedList ();
157
- AtomicInteger nextId = new AtomicInteger (1 );
158
- return visitAvroSchemaToBuildType (schema , visited , true , nextId );
167
+ Deque <String > visited = new LinkedList <> ();
168
+ AtomicInteger nextId = new AtomicInteger (0 );
169
+ return visitAvroSchemaToBuildType (schema , visited , "" , nextId , existingNameToPositions );
159
170
}
160
171
161
172
private static void checkNullType (Type fieldType , String fieldName , Deque <String > visited ) {
@@ -172,11 +183,11 @@ private static void checkNullType(Type fieldType, String fieldName, Deque<String
172
183
throw new HoodieNullSchemaTypeException (sb .toString ());
173
184
} else if (fieldType .typeId () == Type .TypeID .ARRAY ) {
174
185
visited .push (fieldName );
175
- checkNullType (((Types .ArrayType ) fieldType ).elementType (), "element" , visited );
186
+ checkNullType (((Types .ArrayType ) fieldType ).elementType (), InternalSchema . ARRAY_ELEMENT , visited );
176
187
visited .pop ();
177
188
} else if (fieldType .typeId () == Type .TypeID .MAP ) {
178
189
visited .push (fieldName );
179
- checkNullType (((Types .MapType ) fieldType ).valueType (), "value" , visited );
190
+ checkNullType (((Types .MapType ) fieldType ).valueType (), InternalSchema . MAP_VALUE , visited );
180
191
visited .pop ();
181
192
}
182
193
}
@@ -186,28 +197,27 @@ private static void checkNullType(Type fieldType, String fieldName, Deque<String
186
197
*
187
198
* @param schema a avro schema.
188
199
* @param visited track the visit node when do traversal for avro schema; used to check if the name of avro record schema is correct.
189
- * @param firstVisitRoot track whether the current visited schema node is a root node .
200
+ * @param currentFieldPath the dot-separated path to the current field; empty at the root and always ends in a '.' otherwise for ease of concatenation .
190
201
* @param nextId an initial id which used to create id for all fields.
191
202
* @return a hudi type match avro schema.
192
203
*/
193
- private static Type visitAvroSchemaToBuildType (Schema schema , Deque <String > visited , Boolean firstVisitRoot , AtomicInteger nextId ) {
204
+ private static Type visitAvroSchemaToBuildType (Schema schema , Deque <String > visited , String currentFieldPath , AtomicInteger nextId , Map < String , Integer > existingNameToPosition ) {
194
205
switch (schema .getType ()) {
195
206
case RECORD :
196
207
String name = schema .getFullName ();
197
208
if (visited .contains (name )) {
198
209
throw new HoodieSchemaException (String .format ("cannot convert recursive avro record %s" , name ));
199
210
}
200
211
visited .push (name );
201
- List <Schema .Field > fields = schema .getFields ();
212
+ List <Schema .Field > fields = existingNameToPosition .isEmpty () ? schema .getFields () :
213
+ schema .getFields ().stream ()
214
+ .sorted (Comparator .comparing (field -> existingNameToPosition .getOrDefault (currentFieldPath + field .name (), Integer .MAX_VALUE )))
215
+ .collect (Collectors .toList ());
202
216
List <Type > fieldTypes = new ArrayList <>(fields .size ());
203
217
int nextAssignId = nextId .get ();
204
- // when first visit root record, set nextAssignId = 0;
205
- if (firstVisitRoot ) {
206
- nextAssignId = 0 ;
207
- }
208
218
nextId .set (nextAssignId + fields .size ());
209
- fields .stream (). forEach (field -> {
210
- Type fieldType = visitAvroSchemaToBuildType (field .schema (), visited , false , nextId );
219
+ fields .forEach (field -> {
220
+ Type fieldType = visitAvroSchemaToBuildType (field .schema (), visited , currentFieldPath + field . name () + "." , nextId , existingNameToPosition );
211
221
checkNullType (fieldType , field .name (), visited );
212
222
fieldTypes .add (fieldType );
213
223
});
@@ -226,22 +236,24 @@ private static Type visitAvroSchemaToBuildType(Schema schema, Deque<String> visi
226
236
// them up into namespace/struct-name pair)
227
237
return Types .RecordType .get (internalFields , schema .getFullName ());
228
238
case UNION :
229
- List <Type > fTypes = new ArrayList <>();
230
- schema .getTypes ().stream (). forEach (t -> {
231
- fTypes .add (visitAvroSchemaToBuildType (t , visited , false , nextId ));
239
+ List <Type > fTypes = new ArrayList <>(2 );
240
+ schema .getTypes ().forEach (t -> {
241
+ fTypes .add (visitAvroSchemaToBuildType (t , visited , currentFieldPath , nextId , existingNameToPosition ));
232
242
});
233
243
return fTypes .get (0 ) == null ? fTypes .get (1 ) : fTypes .get (0 );
234
244
case ARRAY :
245
+ String elementPath = currentFieldPath + InternalSchema .ARRAY_ELEMENT + "." ;
235
246
Schema elementSchema = schema .getElementType ();
236
247
int elementId = nextId .get ();
237
248
nextId .set (elementId + 1 );
238
- Type elementType = visitAvroSchemaToBuildType (elementSchema , visited , false , nextId );
249
+ Type elementType = visitAvroSchemaToBuildType (elementSchema , visited , elementPath , nextId , existingNameToPosition );
239
250
return Types .ArrayType .get (elementId , AvroInternalSchemaConverter .isOptional (schema .getElementType ()), elementType );
240
251
case MAP :
241
252
int keyId = nextId .get ();
242
253
int valueId = keyId + 1 ;
243
254
nextId .set (valueId + 1 );
244
- Type valueType = visitAvroSchemaToBuildType (schema .getValueType (), visited , false , nextId );
255
+ String valuePath = currentFieldPath + InternalSchema .MAP_VALUE + "." ;
256
+ Type valueType = visitAvroSchemaToBuildType (schema .getValueType (), visited , valuePath , nextId , existingNameToPosition );
245
257
return Types .MapType .get (keyId , valueId , Types .StringType .get (), valueType , AvroInternalSchemaConverter .isOptional (schema .getValueType ()));
246
258
default :
247
259
return visitAvroPrimitiveToBuildInternalType (schema );
0 commit comments