[HUDI-298] Fix issue with incorrect column mapping casusing bad data, during on-the-fly merge of Real Time tables (#956)
* Fix issue with incorrect column mapping casusing bad data, during on-the-fly merge of Real Time tables
This commit is contained in:
committed by
Balaji Varadarajan
parent
c052167c06
commit
12523c379f
@@ -27,6 +27,7 @@ import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.GenericRecordBuilder;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
@@ -82,12 +83,23 @@ public class HoodieMergeOnReadTestUtils {
|
||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
||||
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
|
||||
Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
|
||||
String hiveColumnNames = fields.stream().filter(field -> !field.name().equalsIgnoreCase("datestr"))
|
||||
.map(Schema.Field::name).collect(Collectors.joining(","));
|
||||
hiveColumnNames = hiveColumnNames + ",datestr";
|
||||
|
||||
String hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES);
|
||||
hiveColumnTypes = hiveColumnTypes + ",string";
|
||||
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
|
||||
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypes);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||
jobConf.set("partition_columns", "datestr");
|
||||
jobConf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr");
|
||||
conf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||
conf.set("partition_columns", "datestr");
|
||||
conf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr");
|
||||
conf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypes);
|
||||
inputFormat.setConf(conf);
|
||||
jobConf.addResource(conf);
|
||||
}
|
||||
|
||||
@@ -78,6 +78,7 @@ public class HoodieTestDataGenerator {
|
||||
+ "{\"name\": \"begin_lat\", \"type\": \"double\"}," + "{\"name\": \"begin_lon\", \"type\": \"double\"},"
|
||||
+ "{\"name\": \"end_lat\", \"type\": \"double\"}," + "{\"name\": \"end_lon\", \"type\": \"double\"},"
|
||||
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
|
||||
public static String TRIP_HIVE_COLUMN_TYPES = "double,string,string,string,double,double,double,double,double";
|
||||
public static Schema avroSchema = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
|
||||
public static Schema avroSchemaWithMetadataFields = HoodieAvroUtils.addMetadataFields(avroSchema);
|
||||
|
||||
|
||||
Reference in New Issue
Block a user