[HUDI-3921] Fixed schema evolution cannot work with HUDI-3855 (#5376)
- when columns names are renamed (schema evolution enabled), while copying records from old data file with HoodieMergeHande, renamed columns wasn't handled well.
This commit is contained in:
@@ -70,6 +70,8 @@ import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Deque;
|
||||
import java.util.LinkedList;
|
||||
import java.util.TimeZone;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@@ -405,6 +407,14 @@ public class HoodieAvroUtils {
|
||||
return newRecord;
|
||||
}
|
||||
|
||||
// TODO Unify the logical of rewriteRecordWithMetadata and rewriteEvolutionRecordWithMetadata, and delete this function.
|
||||
public static GenericRecord rewriteEvolutionRecordWithMetadata(GenericRecord genericRecord, Schema newSchema, String fileName) {
|
||||
GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(genericRecord, newSchema, new HashMap<>());
|
||||
// do not preserve FILENAME_METADATA_FIELD
|
||||
newRecord.put(HoodieRecord.FILENAME_METADATA_FIELD_POS, fileName);
|
||||
return newRecord;
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts list of {@link GenericRecord} provided into the {@link GenericRecord} adhering to the
|
||||
* provided {@code newSchema}.
|
||||
@@ -719,14 +729,28 @@ public class HoodieAvroUtils {
|
||||
*
|
||||
* @param oldRecord oldRecord to be rewritten
|
||||
* @param newSchema newSchema used to rewrite oldRecord
|
||||
* @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
|
||||
* @return newRecord for new Schema
|
||||
*/
|
||||
public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, Schema newSchema) {
|
||||
Object newRecord = rewriteRecordWithNewSchema(oldRecord, oldRecord.getSchema(), newSchema);
|
||||
public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, Schema newSchema, Map<String, String> renameCols) {
|
||||
Object newRecord = rewriteRecordWithNewSchema(oldRecord, oldRecord.getSchema(), newSchema, renameCols, new LinkedList<>());
|
||||
return (GenericData.Record) newRecord;
|
||||
}
|
||||
|
||||
private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSchema, Schema newSchema) {
|
||||
/**
|
||||
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new schema.
|
||||
* support deep rewrite for nested record and adjust rename operation.
|
||||
* This particular method does the following things :
|
||||
* a) Create a new empty GenericRecord with the new schema.
|
||||
* b) For GenericRecord, copy over the data from the old schema to the new schema or set default values for all fields of this transformed schema
|
||||
*
|
||||
* @param oldRecord oldRecord to be rewritten
|
||||
* @param newSchema newSchema used to rewrite oldRecord
|
||||
* @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
|
||||
* @param fieldNames track the full name of visited field when we travel new schema.
|
||||
* @return newRecord for new Schema
|
||||
*/
|
||||
private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSchema, Schema newSchema, Map<String, String> renameCols, Deque<String> fieldNames) {
|
||||
if (oldRecord == null) {
|
||||
return null;
|
||||
}
|
||||
@@ -741,10 +765,23 @@ public class HoodieAvroUtils {
|
||||
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
Schema.Field field = fields.get(i);
|
||||
String fieldName = field.name();
|
||||
fieldNames.push(fieldName);
|
||||
if (oldSchema.getField(field.name()) != null) {
|
||||
Schema.Field oldField = oldSchema.getField(field.name());
|
||||
helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema()));
|
||||
helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames));
|
||||
} else {
|
||||
String fieldFullName = createFullName(fieldNames);
|
||||
String[] colNamePartsFromOldSchema = renameCols.getOrDefault(fieldFullName, "").split("\\.");
|
||||
String lastColNameFromOldSchema = colNamePartsFromOldSchema[colNamePartsFromOldSchema.length - 1];
|
||||
// deal with rename
|
||||
if (oldSchema.getField(field.name()) == null && oldSchema.getField(lastColNameFromOldSchema) != null) {
|
||||
// find rename
|
||||
Schema.Field oldField = oldSchema.getField(lastColNameFromOldSchema);
|
||||
helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames));
|
||||
}
|
||||
}
|
||||
fieldNames.pop();
|
||||
}
|
||||
GenericData.Record newRecord = new GenericData.Record(newSchema);
|
||||
for (int i = 0; i < fields.size(); i++) {
|
||||
@@ -765,9 +802,11 @@ public class HoodieAvroUtils {
|
||||
}
|
||||
Collection array = (Collection)oldRecord;
|
||||
List<Object> newArray = new ArrayList();
|
||||
fieldNames.push("element");
|
||||
for (Object element : array) {
|
||||
newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType()));
|
||||
newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType(), renameCols, fieldNames));
|
||||
}
|
||||
fieldNames.pop();
|
||||
return newArray;
|
||||
case MAP:
|
||||
if (!(oldRecord instanceof Map)) {
|
||||
@@ -775,17 +814,29 @@ public class HoodieAvroUtils {
|
||||
}
|
||||
Map<Object, Object> map = (Map<Object, Object>) oldRecord;
|
||||
Map<Object, Object> newMap = new HashMap<>();
|
||||
fieldNames.push("value");
|
||||
for (Map.Entry<Object, Object> entry : map.entrySet()) {
|
||||
newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType()));
|
||||
newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType(), renameCols, fieldNames));
|
||||
}
|
||||
fieldNames.pop();
|
||||
return newMap;
|
||||
case UNION:
|
||||
return rewriteRecordWithNewSchema(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord));
|
||||
return rewriteRecordWithNewSchema(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord), renameCols, fieldNames);
|
||||
default:
|
||||
return rewritePrimaryType(oldRecord, oldSchema, newSchema);
|
||||
}
|
||||
}
|
||||
|
||||
private static String createFullName(Deque<String> fieldNames) {
|
||||
String result = "";
|
||||
if (!fieldNames.isEmpty()) {
|
||||
List<String> parentNames = new ArrayList<>();
|
||||
fieldNames.descendingIterator().forEachRemaining(parentNames::add);
|
||||
result = parentNames.stream().collect(Collectors.joining("."));
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Schema newSchema) {
|
||||
Schema realOldSchema = oldSchema;
|
||||
if (realOldSchema.getType() == UNION) {
|
||||
@@ -958,9 +1009,10 @@ public class HoodieAvroUtils {
|
||||
*
|
||||
* @param oldRecords oldRecords to be rewrite
|
||||
* @param newSchema newSchema used to rewrite oldRecord
|
||||
* @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
|
||||
* @return a iterator of rewrote GeneriRcords
|
||||
*/
|
||||
public static Iterator<GenericRecord> rewriteRecordWithNewSchema(Iterator<GenericRecord> oldRecords, Schema newSchema) {
|
||||
public static Iterator<GenericRecord> rewriteRecordWithNewSchema(Iterator<GenericRecord> oldRecords, Schema newSchema, Map<String, String> renameCols) {
|
||||
if (oldRecords == null || newSchema == null) {
|
||||
return Collections.emptyIterator();
|
||||
}
|
||||
@@ -972,7 +1024,7 @@ public class HoodieAvroUtils {
|
||||
|
||||
@Override
|
||||
public GenericRecord next() {
|
||||
return rewriteRecordWithNewSchema(oldRecords.next(), newSchema);
|
||||
return rewriteRecordWithNewSchema(oldRecords.next(), newSchema, renameCols);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@@ -58,6 +58,7 @@ import java.io.IOException;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Arrays;
|
||||
import java.util.Deque;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
@@ -379,7 +380,7 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
Option<Schema> schemaOption = getMergedSchema(dataBlock);
|
||||
while (recordIterator.hasNext()) {
|
||||
IndexedRecord currentRecord = recordIterator.next();
|
||||
IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get()) : currentRecord;
|
||||
IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get(), new HashMap<>()) : currentRecord;
|
||||
processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN,
|
||||
this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName));
|
||||
totalLogRecords.incrementAndGet();
|
||||
|
||||
@@ -48,6 +48,25 @@ public class InternalSchemaMerger {
|
||||
// we can pass decimalType to reWriteRecordWithNewSchema directly, everything is ok.
|
||||
private boolean useColumnTypeFromFileSchema = true;
|
||||
|
||||
// deal with rename
|
||||
// Whether to use column name from file schema to read files when we find some column name has changed.
|
||||
// spark parquetReader need the original column name to read data, otherwise the parquetReader will read nothing.
|
||||
// eg: current column name is colOldName, now we rename it to colNewName,
|
||||
// we should not pass colNewName to parquetReader, we must pass colOldName to it; when we read out the data.
|
||||
// for log reader
|
||||
// since our reWriteRecordWithNewSchema function support rewrite directly, so we no need this parameter
|
||||
// eg: current column name is colOldName, now we rename it to colNewName,
|
||||
// we can pass colNewName to reWriteRecordWithNewSchema directly, everything is ok.
|
||||
private boolean useColNameFromFileSchema = true;
|
||||
|
||||
public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema, boolean useColNameFromFileSchema) {
|
||||
this.fileSchema = fileSchema;
|
||||
this.querySchema = querySchema;
|
||||
this.ignoreRequiredAttribute = ignoreRequiredAttribute;
|
||||
this.useColumnTypeFromFileSchema = useColumnTypeFromFileSchema;
|
||||
this.useColNameFromFileSchema = useColNameFromFileSchema;
|
||||
}
|
||||
|
||||
public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema) {
|
||||
this.fileSchema = fileSchema;
|
||||
this.querySchema = querySchema;
|
||||
@@ -131,12 +150,15 @@ public class InternalSchemaMerger {
|
||||
private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldField) {
|
||||
Types.Field fieldFromFileSchema = fileSchema.findField(fieldId);
|
||||
String nameFromFileSchema = fieldFromFileSchema.name();
|
||||
String nameFromQuerySchema = querySchema.findField(fieldId).name();
|
||||
Type typeFromFileSchema = fieldFromFileSchema.type();
|
||||
// Current design mechanism guarantees nestedType change is not allowed, so no need to consider.
|
||||
if (newType.isNestedType()) {
|
||||
return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, newType, oldField.doc());
|
||||
return Types.Field.get(oldField.fieldId(), oldField.isOptional(),
|
||||
useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema, newType, oldField.doc());
|
||||
} else {
|
||||
return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc());
|
||||
return Types.Field.get(oldField.fieldId(), oldField.isOptional(),
|
||||
useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -267,4 +267,20 @@ public class InternalSchemaUtils {
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Try to find all renamed cols between oldSchema and newSchema.
|
||||
*
|
||||
* @param oldSchema oldSchema
|
||||
* @param newSchema newSchema which modified from oldSchema
|
||||
* @return renameCols Map. (k, v) -> (colNameFromNewSchema, colNameFromOldSchema)
|
||||
*/
|
||||
public static Map<String, String> collectRenameCols(InternalSchema oldSchema, InternalSchema newSchema) {
|
||||
List<String> colNamesFromWriteSchema = oldSchema.getAllColsFullName();
|
||||
return colNamesFromWriteSchema.stream().filter(f -> {
|
||||
int filedIdFromWriteSchema = oldSchema.findIdByName(f);
|
||||
// try to find the cols which has the same id, but have different colName;
|
||||
return newSchema.getAllIds().contains(filedIdFromWriteSchema) && !newSchema.findfullName(filedIdFromWriteSchema).equalsIgnoreCase(f);
|
||||
}).collect(Collectors.toMap(e -> newSchema.findfullName(oldSchema.findIdByName(e)), e -> e));
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user