Fix addMetadataFields() to carry over 'props'
This commit is contained in:
committed by
vinoth chandar
parent
48aa026dc4
commit
d904fe69ca
@@ -27,6 +27,7 @@ import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.zip.DeflaterOutputStream;
|
||||
import java.util.zip.InflaterInputStream;
|
||||
@@ -40,6 +41,7 @@ import org.apache.avro.io.BinaryEncoder;
|
||||
import org.apache.avro.io.Decoder;
|
||||
import org.apache.avro.io.DecoderFactory;
|
||||
import org.apache.avro.io.EncoderFactory;
|
||||
import org.codehaus.jackson.JsonNode;
|
||||
|
||||
/**
|
||||
* Helper class to do common stuff across Avro.
|
||||
@@ -108,7 +110,11 @@ public class HoodieAvroUtils {
|
||||
parentFields.add(fileNameField);
|
||||
for (Schema.Field field : schema.getFields()) {
|
||||
if (!isMetadataField(field.name())) {
|
||||
parentFields.add(new Schema.Field(field.name(), field.schema(), field.doc(), null));
|
||||
Schema.Field newField = new Schema.Field(field.name(), field.schema(), field.doc(), null);
|
||||
for (Map.Entry<String, JsonNode> prop : field.getJsonProps().entrySet()) {
|
||||
newField.addProp(prop.getKey(), prop.getValue());
|
||||
}
|
||||
parentFields.add(newField);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user