Skip to content

Commit 037f89e

Browse files
authored
[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.
1 parent de5fa1f commit 037f89e

8 files changed

Lines changed: 137 additions & 39 deletions

File tree

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,9 @@
4646
import java.io.IOException;
4747
import java.util.Collections;
4848
import java.util.List;
49+
import java.util.HashMap;
50+
51+
import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty;
4952

5053
/**
5154
* Base class for all write operations logically performed at the file group level.
@@ -98,6 +101,8 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
98101
protected final String fileId;
99102
protected final String writeToken;
100103
protected final TaskContextSupplier taskContextSupplier;
104+
// For full schema evolution
105+
protected final boolean schemaOnReadEnabled;
101106

102107
public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath,
103108
String fileId, HoodieTable<T, I, K, O> hoodieTable, TaskContextSupplier taskContextSupplier) {
@@ -120,6 +125,7 @@ protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String
120125
!hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
121126
this.taskContextSupplier = taskContextSupplier;
122127
this.writeToken = makeWriteToken();
128+
schemaOnReadEnabled = !isNullOrEmpty(hoodieTable.getConfig().getInternalSchema());
123129
}
124130

125131
/**
@@ -224,11 +230,13 @@ public void write(HoodieRecord record, Option<IndexedRecord> avroRecord, Option<
224230
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
225231
*/
226232
protected GenericRecord rewriteRecord(GenericRecord record) {
227-
return HoodieAvroUtils.rewriteRecord(record, writeSchemaWithMetaFields);
233+
return schemaOnReadEnabled ? HoodieAvroUtils.rewriteRecordWithNewSchema(record, writeSchemaWithMetaFields, new HashMap<>())
234+
: HoodieAvroUtils.rewriteRecord(record, writeSchemaWithMetaFields);
228235
}
229236

230237
protected GenericRecord rewriteRecordWithMetadata(GenericRecord record, String fileName) {
231-
return HoodieAvroUtils.rewriteRecordWithMetadata(record, writeSchemaWithMetaFields, fileName);
238+
return schemaOnReadEnabled ? HoodieAvroUtils.rewriteEvolutionRecordWithMetadata(record, writeSchemaWithMetaFields, fileName)
239+
: HoodieAvroUtils.rewriteRecordWithMetadata(record, writeSchemaWithMetaFields, fileName);
232240
}
233241

234242
public abstract List<WriteStatus> close();

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@
3636
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
3737
import org.apache.hudi.internal.schema.utils.AvroSchemaEvolutionUtils;
3838
import org.apache.hudi.internal.schema.utils.SerDeHelper;
39+
import org.apache.hudi.internal.schema.utils.InternalSchemaUtils;
3940
import org.apache.hudi.io.HoodieMergeHandle;
4041
import org.apache.hudi.io.storage.HoodieFileReader;
4142
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
@@ -52,6 +53,8 @@
5253
import java.io.IOException;
5354
import java.util.Iterator;
5455
import java.util.List;
56+
import java.util.HashMap;
57+
import java.util.Map;
5558
import java.util.stream.Collectors;
5659

5760
public class HoodieMergeHelper<T extends HoodieRecordPayload> extends
@@ -93,6 +96,7 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
9396

9497
Option<InternalSchema> querySchemaOpt = SerDeHelper.fromJson(table.getConfig().getInternalSchema());
9598
boolean needToReWriteRecord = false;
99+
Map<String, String> renameCols = new HashMap<>();
96100
// TODO support bootstrap
97101
if (querySchemaOpt.isPresent() && !baseFile.getBootstrapBaseFile().isPresent()) {
98102
// check implicitly add columns, and position reorder(spark sql may change cols order)
@@ -109,10 +113,14 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
109113
&& writeInternalSchema.findIdByName(f) == querySchema.findIdByName(f)
110114
&& writeInternalSchema.findIdByName(f) != -1
111115
&& writeInternalSchema.findType(writeInternalSchema.findIdByName(f)).equals(querySchema.findType(writeInternalSchema.findIdByName(f)))).collect(Collectors.toList());
112-
readSchema = AvroInternalSchemaConverter.convert(new InternalSchemaMerger(writeInternalSchema, querySchema, true, false).mergeSchema(), readSchema.getName());
116+
readSchema = AvroInternalSchemaConverter
117+
.convert(new InternalSchemaMerger(writeInternalSchema, querySchema, true, false, false).mergeSchema(), readSchema.getName());
113118
Schema writeSchemaFromFile = AvroInternalSchemaConverter.convert(writeInternalSchema, readSchema.getName());
114119
needToReWriteRecord = sameCols.size() != colNamesFromWriteSchema.size()
115-
|| SchemaCompatibility.checkReaderWriterCompatibility(writeSchemaFromFile, readSchema).getType() == org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE;
120+
|| SchemaCompatibility.checkReaderWriterCompatibility(readSchema, writeSchemaFromFile).getType() == org.apache.avro.SchemaCompatibility.SchemaCompatibilityType.COMPATIBLE;
121+
if (needToReWriteRecord) {
122+
renameCols = InternalSchemaUtils.collectRenameCols(writeInternalSchema, querySchema);
123+
}
116124
}
117125

118126
try {
@@ -121,7 +129,7 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
121129
readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
122130
} else {
123131
if (needToReWriteRecord) {
124-
readerIterator = HoodieAvroUtils.rewriteRecordWithNewSchema(reader.getRecordIterator(), readSchema);
132+
readerIterator = HoodieAvroUtils.rewriteRecordWithNewSchema(reader.getRecordIterator(), readSchema, renameCols);
125133
} else {
126134
readerIterator = reader.getRecordIterator(readSchema);
127135
}

hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java

Lines changed: 61 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,8 @@
7070
import java.util.Iterator;
7171
import java.util.List;
7272
import java.util.Map;
73+
import java.util.Deque;
74+
import java.util.LinkedList;
7375
import java.util.TimeZone;
7476
import java.util.stream.Collectors;
7577

@@ -405,6 +407,14 @@ public static GenericRecord rewriteRecordWithMetadata(GenericRecord genericRecor
405407
return newRecord;
406408
}
407409

410+
// TODO Unify the logical of rewriteRecordWithMetadata and rewriteEvolutionRecordWithMetadata, and delete this function.
411+
public static GenericRecord rewriteEvolutionRecordWithMetadata(GenericRecord genericRecord, Schema newSchema, String fileName) {
412+
GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(genericRecord, newSchema, new HashMap<>());
413+
// do not preserve FILENAME_METADATA_FIELD
414+
newRecord.put(HoodieRecord.FILENAME_METADATA_FIELD_POS, fileName);
415+
return newRecord;
416+
}
417+
408418
/**
409419
* Converts list of {@link GenericRecord} provided into the {@link GenericRecord} adhering to the
410420
* provided {@code newSchema}.
@@ -719,14 +729,28 @@ public static Object getRecordColumnValues(HoodieRecord<? extends HoodieRecordPa
719729
*
720730
* @param oldRecord oldRecord to be rewritten
721731
* @param newSchema newSchema used to rewrite oldRecord
732+
* @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
722733
* @return newRecord for new Schema
723734
*/
724-
public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, Schema newSchema) {
725-
Object newRecord = rewriteRecordWithNewSchema(oldRecord, oldRecord.getSchema(), newSchema);
735+
public static GenericRecord rewriteRecordWithNewSchema(IndexedRecord oldRecord, Schema newSchema, Map<String, String> renameCols) {
736+
Object newRecord = rewriteRecordWithNewSchema(oldRecord, oldRecord.getSchema(), newSchema, renameCols, new LinkedList<>());
726737
return (GenericData.Record) newRecord;
727738
}
728739

729-
private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSchema, Schema newSchema) {
740+
/**
741+
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new schema.
742+
* support deep rewrite for nested record and adjust rename operation.
743+
* This particular method does the following things :
744+
* a) Create a new empty GenericRecord with the new schema.
745+
* 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
746+
*
747+
* @param oldRecord oldRecord to be rewritten
748+
* @param newSchema newSchema used to rewrite oldRecord
749+
* @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
750+
* @param fieldNames track the full name of visited field when we travel new schema.
751+
* @return newRecord for new Schema
752+
*/
753+
private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSchema, Schema newSchema, Map<String, String> renameCols, Deque<String> fieldNames) {
730754
if (oldRecord == null) {
731755
return null;
732756
}
@@ -741,10 +765,23 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSch
741765

742766
for (int i = 0; i < fields.size(); i++) {
743767
Schema.Field field = fields.get(i);
768+
String fieldName = field.name();
769+
fieldNames.push(fieldName);
744770
if (oldSchema.getField(field.name()) != null) {
745771
Schema.Field oldField = oldSchema.getField(field.name());
746-
helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema()));
772+
helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames));
773+
} else {
774+
String fieldFullName = createFullName(fieldNames);
775+
String[] colNamePartsFromOldSchema = renameCols.getOrDefault(fieldFullName, "").split("\\.");
776+
String lastColNameFromOldSchema = colNamePartsFromOldSchema[colNamePartsFromOldSchema.length - 1];
777+
// deal with rename
778+
if (oldSchema.getField(field.name()) == null && oldSchema.getField(lastColNameFromOldSchema) != null) {
779+
// find rename
780+
Schema.Field oldField = oldSchema.getField(lastColNameFromOldSchema);
781+
helper.put(i, rewriteRecordWithNewSchema(indexedRecord.get(oldField.pos()), oldField.schema(), fields.get(i).schema(), renameCols, fieldNames));
782+
}
747783
}
784+
fieldNames.pop();
748785
}
749786
GenericData.Record newRecord = new GenericData.Record(newSchema);
750787
for (int i = 0; i < fields.size(); i++) {
@@ -765,27 +802,41 @@ private static Object rewriteRecordWithNewSchema(Object oldRecord, Schema oldSch
765802
}
766803
Collection array = (Collection)oldRecord;
767804
List<Object> newArray = new ArrayList();
805+
fieldNames.push("element");
768806
for (Object element : array) {
769-
newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType()));
807+
newArray.add(rewriteRecordWithNewSchema(element, oldSchema.getElementType(), newSchema.getElementType(), renameCols, fieldNames));
770808
}
809+
fieldNames.pop();
771810
return newArray;
772811
case MAP:
773812
if (!(oldRecord instanceof Map)) {
774813
throw new IllegalArgumentException("cannot rewrite record with different type");
775814
}
776815
Map<Object, Object> map = (Map<Object, Object>) oldRecord;
777816
Map<Object, Object> newMap = new HashMap<>();
817+
fieldNames.push("value");
778818
for (Map.Entry<Object, Object> entry : map.entrySet()) {
779-
newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType()));
819+
newMap.put(entry.getKey(), rewriteRecordWithNewSchema(entry.getValue(), oldSchema.getValueType(), newSchema.getValueType(), renameCols, fieldNames));
780820
}
821+
fieldNames.pop();
781822
return newMap;
782823
case UNION:
783-
return rewriteRecordWithNewSchema(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord));
824+
return rewriteRecordWithNewSchema(oldRecord, getActualSchemaFromUnion(oldSchema, oldRecord), getActualSchemaFromUnion(newSchema, oldRecord), renameCols, fieldNames);
784825
default:
785826
return rewritePrimaryType(oldRecord, oldSchema, newSchema);
786827
}
787828
}
788829

830+
private static String createFullName(Deque<String> fieldNames) {
831+
String result = "";
832+
if (!fieldNames.isEmpty()) {
833+
List<String> parentNames = new ArrayList<>();
834+
fieldNames.descendingIterator().forEachRemaining(parentNames::add);
835+
result = parentNames.stream().collect(Collectors.joining("."));
836+
}
837+
return result;
838+
}
839+
789840
private static Object rewritePrimaryType(Object oldValue, Schema oldSchema, Schema newSchema) {
790841
Schema realOldSchema = oldSchema;
791842
if (realOldSchema.getType() == UNION) {
@@ -958,9 +1009,10 @@ private static Schema getActualSchemaFromUnion(Schema schema, Object data) {
9581009
*
9591010
* @param oldRecords oldRecords to be rewrite
9601011
* @param newSchema newSchema used to rewrite oldRecord
1012+
* @param renameCols a map store all rename cols, (k, v)-> (colNameFromNewSchema, colNameFromOldSchema)
9611013
* @return a iterator of rewrote GeneriRcords
9621014
*/
963-
public static Iterator<GenericRecord> rewriteRecordWithNewSchema(Iterator<GenericRecord> oldRecords, Schema newSchema) {
1015+
public static Iterator<GenericRecord> rewriteRecordWithNewSchema(Iterator<GenericRecord> oldRecords, Schema newSchema, Map<String, String> renameCols) {
9641016
if (oldRecords == null || newSchema == null) {
9651017
return Collections.emptyIterator();
9661018
}
@@ -972,7 +1024,7 @@ public boolean hasNext() {
9721024

9731025
@Override
9741026
public GenericRecord next() {
975-
return rewriteRecordWithNewSchema(oldRecords.next(), newSchema);
1027+
return rewriteRecordWithNewSchema(oldRecords.next(), newSchema, renameCols);
9761028
}
9771029
};
9781030
}

hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,7 @@
5858
import java.util.ArrayDeque;
5959
import java.util.Arrays;
6060
import java.util.Deque;
61+
import java.util.HashMap;
6162
import java.util.HashSet;
6263
import java.util.List;
6364
import java.util.Set;
@@ -379,7 +380,7 @@ private void processDataBlock(HoodieDataBlock dataBlock, Option<KeySpec> keySpec
379380
Option<Schema> schemaOption = getMergedSchema(dataBlock);
380381
while (recordIterator.hasNext()) {
381382
IndexedRecord currentRecord = recordIterator.next();
382-
IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get()) : currentRecord;
383+
IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get(), new HashMap<>()) : currentRecord;
383384
processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN,
384385
this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName));
385386
totalLogRecords.incrementAndGet();

hudi-common/src/main/java/org/apache/hudi/internal/schema/action/InternalSchemaMerger.java

Lines changed: 24 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,25 @@ public class InternalSchemaMerger {
4848
// we can pass decimalType to reWriteRecordWithNewSchema directly, everything is ok.
4949
private boolean useColumnTypeFromFileSchema = true;
5050

51+
// deal with rename
52+
// Whether to use column name from file schema to read files when we find some column name has changed.
53+
// spark parquetReader need the original column name to read data, otherwise the parquetReader will read nothing.
54+
// eg: current column name is colOldName, now we rename it to colNewName,
55+
// we should not pass colNewName to parquetReader, we must pass colOldName to it; when we read out the data.
56+
// for log reader
57+
// since our reWriteRecordWithNewSchema function support rewrite directly, so we no need this parameter
58+
// eg: current column name is colOldName, now we rename it to colNewName,
59+
// we can pass colNewName to reWriteRecordWithNewSchema directly, everything is ok.
60+
private boolean useColNameFromFileSchema = true;
61+
62+
public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema, boolean useColNameFromFileSchema) {
63+
this.fileSchema = fileSchema;
64+
this.querySchema = querySchema;
65+
this.ignoreRequiredAttribute = ignoreRequiredAttribute;
66+
this.useColumnTypeFromFileSchema = useColumnTypeFromFileSchema;
67+
this.useColNameFromFileSchema = useColNameFromFileSchema;
68+
}
69+
5170
public InternalSchemaMerger(InternalSchema fileSchema, InternalSchema querySchema, boolean ignoreRequiredAttribute, boolean useColumnTypeFromFileSchema) {
5271
this.fileSchema = fileSchema;
5372
this.querySchema = querySchema;
@@ -131,12 +150,15 @@ private List<Types.Field> buildRecordType(List<Types.Field> oldFields, List<Type
131150
private Types.Field dealWithRename(int fieldId, Type newType, Types.Field oldField) {
132151
Types.Field fieldFromFileSchema = fileSchema.findField(fieldId);
133152
String nameFromFileSchema = fieldFromFileSchema.name();
153+
String nameFromQuerySchema = querySchema.findField(fieldId).name();
134154
Type typeFromFileSchema = fieldFromFileSchema.type();
135155
// Current design mechanism guarantees nestedType change is not allowed, so no need to consider.
136156
if (newType.isNestedType()) {
137-
return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, newType, oldField.doc());
157+
return Types.Field.get(oldField.fieldId(), oldField.isOptional(),
158+
useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema, newType, oldField.doc());
138159
} else {
139-
return Types.Field.get(oldField.fieldId(), oldField.isOptional(), nameFromFileSchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc());
160+
return Types.Field.get(oldField.fieldId(), oldField.isOptional(),
161+
useColNameFromFileSchema ? nameFromFileSchema : nameFromQuerySchema, useColumnTypeFromFileSchema ? typeFromFileSchema : newType, oldField.doc());
140162
}
141163
}
142164

hudi-common/src/main/java/org/apache/hudi/internal/schema/utils/InternalSchemaUtils.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -267,4 +267,20 @@ public static String createFullName(String name, Deque<String> fieldNames) {
267267
}
268268
return result;
269269
}
270+
271+
/**
272+
* Try to find all renamed cols between oldSchema and newSchema.
273+
*
274+
* @param oldSchema oldSchema
275+
* @param newSchema newSchema which modified from oldSchema
276+
* @return renameCols Map. (k, v) -> (colNameFromNewSchema, colNameFromOldSchema)
277+
*/
278+
public static Map<String, String> collectRenameCols(InternalSchema oldSchema, InternalSchema newSchema) {
279+
List<String> colNamesFromWriteSchema = oldSchema.getAllColsFullName();
280+
return colNamesFromWriteSchema.stream().filter(f -> {
281+
int filedIdFromWriteSchema = oldSchema.findIdByName(f);
282+
// try to find the cols which has the same id, but have different colName;
283+
return newSchema.getAllIds().contains(filedIdFromWriteSchema) && !newSchema.findfullName(filedIdFromWriteSchema).equalsIgnoreCase(f);
284+
}).collect(Collectors.toMap(e -> newSchema.findfullName(oldSchema.findIdByName(e)), e -> e));
285+
}
270286
}

hudi-common/src/test/java/org/apache/hudi/internal/schema/utils/TestAvroSchemaEvolutionUtils.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -284,7 +284,7 @@ public void testReWriteRecordWithTypeChanged() {
284284
.updateColumnType("col6", Types.StringType.get());
285285
InternalSchema newSchema = SchemaChangeUtils.applyTableChanges2Schema(internalSchema, updateChange);
286286
Schema newAvroSchema = AvroInternalSchemaConverter.convert(newSchema, avroSchema.getName());
287-
GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema);
287+
GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, new HashMap<>());
288288

289289
Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newRecord), true);
290290
}
@@ -349,7 +349,7 @@ public void testReWriteNestRecord() {
349349
);
350350

351351
Schema newAvroSchema = AvroInternalSchemaConverter.convert(newRecord, schema.getName());
352-
GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema);
352+
GenericRecord newAvroRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(avroRecord, newAvroSchema, new HashMap<>());
353353
// test the correctly of rewrite
354354
Assertions.assertEquals(GenericData.get().validate(newAvroSchema, newAvroRecord), true);
355355
}

0 commit comments

Comments
 (0)