1
0

[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:
Udit Mehrotra
2019-10-16 02:05:53 -07:00
committed by Balaji Varadarajan
parent c052167c06
commit 12523c379f
6 changed files with 88 additions and 38 deletions

View File

@@ -36,6 +36,7 @@ import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.io.ArrayWritable;
@@ -76,6 +77,8 @@ import org.junit.rules.TemporaryFolder;
public class HoodieRealtimeRecordReaderTest {
private static final String PARTITION_COLUMN = "datestr";
private JobConf jobConf;
private FileSystem fs;
private Configuration hadoopConf;
@@ -158,7 +161,22 @@ public class HoodieRealtimeRecordReaderTest {
testReader(false);
}
public void testReader(boolean partitioned) throws Exception {
private void setHiveColumnNameProps(List<Schema.Field> fields, JobConf jobConf, boolean isPartitioned) {
String names = fields.stream().map(Field::name).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
String hiveOrderedColumnNames = fields.stream().filter(field -> !field.name().equalsIgnoreCase(PARTITION_COLUMN))
.map(Field::name).collect(Collectors.joining(","));
if (isPartitioned) {
hiveOrderedColumnNames += "," + PARTITION_COLUMN;
jobConf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, PARTITION_COLUMN);
}
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveOrderedColumnNames);
}
private void testReader(boolean partitioned) throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
@@ -213,13 +231,7 @@ public class HoodieRealtimeRecordReaderTest {
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
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(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
if (partitioned) {
jobConf.set("partition_columns", "datestr");
}
setHiveColumnNameProps(fields, jobConf, partitioned);
// validate record reader compaction
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
@@ -277,11 +289,7 @@ public class HoodieRealtimeRecordReaderTest {
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
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(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
setHiveColumnNameProps(fields, jobConf, true);
// Enable merge skipping.
jobConf.set("hoodie.realtime.merge.skip", "true");
@@ -356,12 +364,7 @@ public class HoodieRealtimeRecordReaderTest {
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");
setHiveColumnNameProps(fields, jobConf, true);
// validate record reader compaction
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
@@ -502,11 +505,7 @@ public class HoodieRealtimeRecordReaderTest {
assert (firstSchemaFields.containsAll(fields) == false);
// Try to read all the fields passed by the new schema
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");
setHiveColumnNameProps(fields, jobConf, true);
HoodieRealtimeRecordReader recordReader = null;
try {
@@ -518,11 +517,7 @@ public class HoodieRealtimeRecordReaderTest {
}
// Try to read all the fields passed by the new schema
names = firstSchemaFields.stream().map(f -> f.name()).collect(Collectors.joining(","));
positions = firstSchemaFields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");
setHiveColumnNameProps(firstSchemaFields, jobConf, true);
// This time read only the fields which are part of parquet
recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
// use reader to read base Parquet File and log file