1
0

[HUDI-727]: Copy default values of fields if not present when rewriting incoming record with new schema (#1427)

This commit is contained in:
Pratyaksh Sharma
2020-04-13 06:25:26 +05:30
committed by GitHub
parent 5d717a28f4
commit 6d7ca2cf7e
3 changed files with 84 additions and 17 deletions

View File

@@ -32,7 +32,6 @@ import org.apache.avro.io.BinaryDecoder;
import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.EncoderFactory;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.node.NullNode;
import java.io.ByteArrayInputStream;
@@ -44,6 +43,7 @@ import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -104,15 +104,15 @@ public class HoodieAvroUtils {
List<Schema.Field> parentFields = new ArrayList<>();
Schema.Field commitTimeField =
new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
new Schema.Field(HoodieRecord.COMMIT_TIME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", (Object) null);
Schema.Field commitSeqnoField =
new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
new Schema.Field(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", (Object) null);
Schema.Field recordKeyField =
new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", (Object) null);
Schema.Field partitionPathField =
new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", (Object) null);
Schema.Field fileNameField =
new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
new Schema.Field(HoodieRecord.FILENAME_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", (Object) null);
parentFields.add(commitTimeField);
parentFields.add(commitSeqnoField);
@@ -121,8 +121,8 @@ public class HoodieAvroUtils {
parentFields.add(fileNameField);
for (Schema.Field field : schema.getFields()) {
if (!isMetadataField(field.name())) {
Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue());
for (Map.Entry<String, JsonNode> prop : field.getJsonProps().entrySet()) {
Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultVal());
for (Map.Entry<String, Object> prop : field.getObjectProps().entrySet()) {
newField.addProp(prop.getKey(), prop.getValue());
}
parentFields.add(newField);
@@ -191,7 +191,7 @@ public class HoodieAvroUtils {
* schema.
*/
public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema) {
return rewrite(record, record.getSchema(), newSchema);
return rewrite(record, getCombinedFieldsToWrite(record.getSchema(), newSchema), newSchema);
}
/**
@@ -199,13 +199,17 @@ public class HoodieAvroUtils {
* schema.
*/
public static GenericRecord rewriteRecordWithOnlyNewSchemaFields(GenericRecord record, Schema newSchema) {
return rewrite(record, newSchema, newSchema);
return rewrite(record, new LinkedHashSet<>(newSchema.getFields()), newSchema);
}
private static GenericRecord rewrite(GenericRecord record, Schema schemaWithFields, Schema newSchema) {
private static GenericRecord rewrite(GenericRecord record, LinkedHashSet<Field> fieldsToWrite, Schema newSchema) {
GenericRecord newRecord = new GenericData.Record(newSchema);
for (Schema.Field f : schemaWithFields.getFields()) {
newRecord.put(f.name(), record.get(f.name()));
for (Schema.Field f : fieldsToWrite) {
if (record.get(f.name()) == null) {
newRecord.put(f.name(), f.defaultVal());
} else {
newRecord.put(f.name(), record.get(f.name()));
}
}
if (!GenericData.get().validate(newSchema, newRecord)) {
throw new SchemaCompatabilityException(
@@ -214,6 +218,19 @@ public class HoodieAvroUtils {
return newRecord;
}
/**
* Generates a super set of fields from both old and new schema.
*/
private static LinkedHashSet<Field> getCombinedFieldsToWrite(Schema oldSchema, Schema newSchema) {
LinkedHashSet<Field> allFields = new LinkedHashSet<>(oldSchema.getFields());
for (Schema.Field f : newSchema.getFields()) {
if (!allFields.contains(f) && !isMetadataField(f.name())) {
allFields.add(f);
}
}
return allFields;
}
public static byte[] compress(String text) {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {