[HUDI-3204] Fixing partition-values being derived from partition-path instead of source columns (#5364)
- Scaffolded `Spark24HoodieParquetFileFormat` extending `ParquetFileFormat` and overriding the behavior of adding partition columns to every row - Amended `SparkAdapter`s `createHoodieParquetFileFormat` API to be able to configure whether to append partition values or not - Fallback to append partition values in cases when the source columns are not persisted in data-file - Fixing HoodieBaseRelation incorrectly handling mandatory columns
This commit is contained in:
@@ -18,13 +18,7 @@
|
||||
|
||||
package org.apache.hudi.hadoop.utils;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
import org.apache.avro.JsonProperties;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericArray;
|
||||
@@ -32,8 +26,8 @@ import org.apache.avro.generic.GenericFixed;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.serde2.io.DoubleWritable;
|
||||
import org.apache.hadoop.hive.serde2.io.DateWritable;
|
||||
import org.apache.hadoop.hive.serde2.io.DoubleWritable;
|
||||
import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
|
||||
import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
|
||||
@@ -46,6 +40,12 @@ import org.apache.hadoop.io.LongWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -60,6 +60,9 @@ import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema;
|
||||
import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;
|
||||
|
||||
public class HoodieRealtimeRecordReaderUtils {
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieRealtimeRecordReaderUtils.class);
|
||||
|
||||
@@ -287,6 +290,14 @@ public class HoodieRealtimeRecordReaderUtils {
|
||||
List<String> fieldsToAdd = partitioningFields.stream().map(String::toLowerCase)
|
||||
.filter(x -> !firstLevelFieldNames.contains(x)).collect(Collectors.toList());
|
||||
|
||||
return HoodieAvroUtils.appendNullSchemaFields(schema, fieldsToAdd);
|
||||
return appendNullSchemaFields(schema, fieldsToAdd);
|
||||
}
|
||||
|
||||
private static Schema appendNullSchemaFields(Schema schema, List<String> newFieldNames) {
|
||||
List<Schema.Field> newFields = new ArrayList<>();
|
||||
for (String newField : newFieldNames) {
|
||||
newFields.add(new Schema.Field(newField, createNullableSchema(Schema.Type.STRING), "", JsonProperties.NULL_VALUE));
|
||||
}
|
||||
return appendFieldsToSchema(schema, newFields);
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user