Skip to content

Commit 13c3728

Browse files
address pr feedback
1 parent 553fe76 commit 13c3728

File tree

2 files changed

+16
-41
lines changed

2 files changed

+16
-41
lines changed

hudi-common/src/main/java/org/apache/hudi/common/schema/HoodieSchema.java

Lines changed: 0 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -501,20 +501,6 @@ public HoodieSchemaType getType() {
501501
return type;
502502
}
503503

504-
/**
505-
* If this is a union schema representing a null, it returns the underlying non-null type.
506-
* @return the underlying non-null HoodieSchema
507-
*/
508-
public HoodieSchema getUnderlyingType() {
509-
if (HoodieSchemaType.UNION == type) {
510-
return getTypes().stream()
511-
.filter(schema -> schema.getType() != HoodieSchemaType.NULL)
512-
.findFirst()
513-
.orElseThrow(() -> new IllegalArgumentException("No non-null type found in Union"));
514-
}
515-
return this;
516-
}
517-
518504
/**
519505
* Returns the name of the schema if a record, otherwise it returns the name of the type.
520506
*

hudi-common/src/main/java/org/apache/hudi/internal/schema/convert/InternalSchemaConverter.java

Lines changed: 16 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -52,17 +52,6 @@
5252
* Converts a HoodieSchema into InternalSchema, or convert InternalSchema to a HoodieSchema.
5353
*/
5454
public class InternalSchemaConverter {
55-
56-
// NOTE: We're using dot as field's name delimiter for nested fields
57-
// so that Avro is able to interpret qualified name as rather
58-
// the combination of the Avro's namespace and actual record's name.
59-
// For example qualified nested field's name "trip.fare.amount",
60-
// Avro will produce a record with
61-
// - Namespace: "trip.fare"
62-
// - Name: "amount"
63-
//
64-
// This is crucial aspect of maintaining compatibility b/w schemas, after
65-
// converting Avro [[Schema]]s to [[InternalSchema]]s and back
6655
private static final String FIELD_NAME_DELIMITER = ".";
6756

6857
/**
@@ -91,37 +80,37 @@ public static InternalSchema pruneHoodieSchemaToInternalSchema(HoodieSchema sche
9180
static List<String> collectColNamesFromSchema(HoodieSchema schema) {
9281
List<String> result = new ArrayList<>();
9382
Deque<String> visited = new LinkedList<>();
94-
collectColNamesFromHoodieSchema(schema, visited, result);
83+
collectColNamesFromSchema(schema, visited, result);
9584
return result;
9685
}
9786

98-
private static void collectColNamesFromHoodieSchema(HoodieSchema schema, Deque<String> visited, List<String> resultSet) {
87+
private static void collectColNamesFromSchema(HoodieSchema schema, Deque<String> visited, List<String> resultSet) {
9988
switch (schema.getType()) {
10089
case RECORD:
10190
List<HoodieSchemaField> fields = schema.getFields();
10291
for (HoodieSchemaField f : fields) {
10392
visited.push(f.name());
104-
collectColNamesFromHoodieSchema(f.schema(), visited, resultSet);
93+
collectColNamesFromSchema(f.schema(), visited, resultSet);
10594
visited.pop();
10695
addFullNameIfLeafNode(f.schema(), f.name(), visited, resultSet);
10796
}
10897
return;
10998

11099
case UNION:
111-
collectColNamesFromHoodieSchema(schema.getUnderlyingType(), visited, resultSet);
100+
collectColNamesFromSchema(schema.getNonNullType(), visited, resultSet);
112101
return;
113102

114103
case ARRAY:
115104
visited.push("element");
116-
collectColNamesFromHoodieSchema(schema.getElementType(), visited, resultSet);
105+
collectColNamesFromSchema(schema.getElementType(), visited, resultSet);
117106
visited.pop();
118107
addFullNameIfLeafNode(schema.getElementType(), "element", visited, resultSet);
119108
return;
120109

121110
case MAP:
122111
addFullNameIfLeafNode(HoodieSchemaType.STRING, "key", visited, resultSet);
123112
visited.push("value");
124-
collectColNamesFromHoodieSchema(schema.getValueType(), visited, resultSet);
113+
collectColNamesFromSchema(schema.getValueType(), visited, resultSet);
125114
visited.pop();
126115
addFullNameIfLeafNode(schema.getValueType(), "value", visited, resultSet);
127116
return;
@@ -131,7 +120,7 @@ private static void collectColNamesFromHoodieSchema(HoodieSchema schema, Deque<S
131120
}
132121

133122
private static void addFullNameIfLeafNode(HoodieSchema schema, String name, Deque<String> visited, List<String> resultSet) {
134-
addFullNameIfLeafNode(schema.getUnderlyingType().getType(), name, visited, resultSet);
123+
addFullNameIfLeafNode(schema.getNonNullType().getType(), name, visited, resultSet);
135124
}
136125

137126
private static void addFullNameIfLeafNode(HoodieSchemaType type, String name, Deque<String> visited, List<String> resultSet) {
@@ -176,7 +165,7 @@ public static HoodieSchema fixNullOrdering(HoodieSchema schema) {
176165
* @return a HoodieSchema.
177166
*/
178167
public static HoodieSchema convert(Types.RecordType type, String name) {
179-
return buildHoodieSchemaFromType(type, name);
168+
return buildSchemaFromType(type, name);
180169
}
181170

182171
/**
@@ -187,7 +176,7 @@ public static HoodieSchema convert(Types.RecordType type, String name) {
187176
* @return a HoodieSchema.
188177
*/
189178
public static HoodieSchema convert(Type type, String name) {
190-
return buildHoodieSchemaFromType(type, name);
179+
return buildSchemaFromType(type, name);
191180
}
192181

193182
/** Convert a HoodieSchema into internal type. */
@@ -218,7 +207,7 @@ public static Type buildTypeFromHoodieSchema(HoodieSchema schema, Map<String, In
218207
// set flag to check this has not been visited.
219208
Deque<String> visited = new LinkedList<>();
220209
AtomicInteger nextId = new AtomicInteger(0);
221-
return visitHoodieSchemaToBuildType(schema, visited, "", nextId, existingNameToPositions);
210+
return visitSchemaToBuildType(schema, visited, "", nextId, existingNameToPositions);
222211
}
223212

224213
private static void checkNullType(Type fieldType, String fieldName, Deque<String> visited) {
@@ -253,7 +242,7 @@ private static void checkNullType(Type fieldType, String fieldName, Deque<String
253242
* @param nextId an initial id which used to create id for all fields.
254243
* @return a hudi type match HoodieSchema.
255244
*/
256-
private static Type visitHoodieSchemaToBuildType(HoodieSchema schema, Deque<String> visited, String currentFieldPath, AtomicInteger nextId, Map<String, Integer> existingNameToPosition) {
245+
private static Type visitSchemaToBuildType(HoodieSchema schema, Deque<String> visited, String currentFieldPath, AtomicInteger nextId, Map<String, Integer> existingNameToPosition) {
257246
switch (schema.getType()) {
258247
case RECORD:
259248
String name = schema.getFullName();
@@ -269,7 +258,7 @@ private static Type visitHoodieSchemaToBuildType(HoodieSchema schema, Deque<Stri
269258
int nextAssignId = nextId.get();
270259
nextId.set(nextAssignId + fields.size());
271260
fields.forEach(field -> {
272-
Type fieldType = visitHoodieSchemaToBuildType(field.schema(), visited, currentFieldPath + field.name() + ".", nextId, existingNameToPosition);
261+
Type fieldType = visitSchemaToBuildType(field.schema(), visited, currentFieldPath + field.name() + ".", nextId, existingNameToPosition);
273262
checkNullType(fieldType, field.name(), visited);
274263
fieldTypes.add(fieldType);
275264
});
@@ -290,22 +279,22 @@ private static Type visitHoodieSchemaToBuildType(HoodieSchema schema, Deque<Stri
290279
case UNION:
291280
List<Type> fTypes = new ArrayList<>(2);
292281
schema.getTypes().forEach(t -> {
293-
fTypes.add(visitHoodieSchemaToBuildType(t, visited, currentFieldPath, nextId, existingNameToPosition));
282+
fTypes.add(visitSchemaToBuildType(t, visited, currentFieldPath, nextId, existingNameToPosition));
294283
});
295284
return fTypes.get(0) == null ? fTypes.get(1) : fTypes.get(0);
296285
case ARRAY:
297286
String elementPath = currentFieldPath + InternalSchema.ARRAY_ELEMENT + ".";
298287
HoodieSchema elementSchema = schema.getElementType();
299288
int elementId = nextId.get();
300289
nextId.set(elementId + 1);
301-
Type elementType = visitHoodieSchemaToBuildType(elementSchema, visited, elementPath, nextId, existingNameToPosition);
290+
Type elementType = visitSchemaToBuildType(elementSchema, visited, elementPath, nextId, existingNameToPosition);
302291
return Types.ArrayType.get(elementId, schema.getElementType().isNullable(), elementType);
303292
case MAP:
304293
int keyId = nextId.get();
305294
int valueId = keyId + 1;
306295
nextId.set(valueId + 1);
307296
String valuePath = currentFieldPath + InternalSchema.MAP_VALUE + ".";
308-
Type valueType = visitHoodieSchemaToBuildType(schema.getValueType(), visited, valuePath, nextId, existingNameToPosition);
297+
Type valueType = visitSchemaToBuildType(schema.getValueType(), visited, valuePath, nextId, existingNameToPosition);
309298
return Types.MapType.get(keyId, valueId, Types.StringType.get(), valueType, schema.getValueType().isNullable());
310299
default:
311300
return visitPrimitiveToBuildInternalType(schema);
@@ -379,7 +368,7 @@ private static Type visitPrimitiveToBuildInternalType(HoodieSchema hoodieSchema)
379368
* @param recordName the record name
380369
* @return a HoodieSchema match this type
381370
*/
382-
private static HoodieSchema buildHoodieSchemaFromType(Type type, String recordName) {
371+
private static HoodieSchema buildSchemaFromType(Type type, String recordName) {
383372
Map<Type, HoodieSchema> cache = new HashMap<>();
384373
return visitInternalSchemaToBuildHoodieSchema(type, cache, recordName);
385374
}

0 commit comments

Comments
 (0)