[HUDI-3838] Implemented drop partition column feature for delta streamer code path (#5294)
* [HUDI-3838] Implemented drop partition column feature for delta streamer code path * Ensure drop partition table config is updated in hoodie.props Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
committed by
GitHub
parent
101b82a679
commit
d16740976e
@@ -441,6 +441,17 @@ public class HoodieAvroUtils {
|
||||
return records.stream().map(r -> rewriteRecord(r, newSchema)).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Given an Avro record and list of columns to remove, this method removes the list of columns from
|
||||
* the given avro record using rewriteRecord method.
|
||||
* <p>
|
||||
* To better understand how it removes please check {@link #rewriteRecord(GenericRecord, Schema)}
|
||||
*/
|
||||
public static GenericRecord removeFields(GenericRecord record, List<String> fieldsToRemove) {
|
||||
Schema newSchema = removeFields(record.getSchema(), fieldsToRemove);
|
||||
return rewriteRecord(record, newSchema);
|
||||
}
|
||||
|
||||
private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
|
||||
Schema oldSchema = oldRecord.getSchema();
|
||||
Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
|
||||
|
||||
Reference in New Issue
Block a user