1
0

[HUDI-1722]Fix hive beeline/spark-sql query specified field on mor table occur NPE (#2722)

This commit is contained in:
xiarixiaoyao
2021-05-12 20:52:37 +08:00
committed by GitHub
parent b98c9ab439
commit 6f7ff7e8ca
4 changed files with 110 additions and 2 deletions

View File

@@ -85,12 +85,12 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
// risk of experiencing race conditions. Hence, we synchronize on the JobConf object here. There is negligible
// latency incurred here due to the synchronization since get record reader is called once per spilt before the
// actual heavy lifting of reading the parquet files happen.
if (jobConf.get(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP) == null) {
if (HoodieRealtimeInputFormatUtils.canAddProjectionToJobConf(realtimeSplit, jobConf)) {
synchronized (jobConf) {
LOG.info(
"Before adding Hoodie columns, Projections :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR)
+ ", Ids :" + jobConf.get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR));
if (jobConf.get(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP) == null) {
if (HoodieRealtimeInputFormatUtils.canAddProjectionToJobConf(realtimeSplit, jobConf)) {
// Hive (across all versions) fails for queries like select count(`_hoodie_commit_time`) from table;
// In this case, the projection fields gets removed. Looking at HiveInputFormat implementation, in some cases
// hoodie additional projection columns are reset after calling setConf and only natural projections

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.hadoop.utils;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.mapred.FileSplit;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.SplitLocationInfo;
import org.apache.hudi.hadoop.realtime.RealtimeSplit;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -218,6 +220,18 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
addProjectionField(configuration, HoodieRecord.PARTITION_PATH_METADATA_FIELD, HoodieInputFormatUtils.HOODIE_PARTITION_PATH_COL_POS);
}
public static boolean requiredProjectionFieldsExistInConf(Configuration configuration) {
String readColNames = configuration.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "");
return readColNames.contains(HoodieRecord.RECORD_KEY_METADATA_FIELD)
&& readColNames.contains(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
&& readColNames.contains(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
}
public static boolean canAddProjectionToJobConf(final RealtimeSplit realtimeSplit, final JobConf jobConf) {
return jobConf.get(HoodieInputFormatUtils.HOODIE_READ_COLUMNS_PROP) == null
|| (!realtimeSplit.getDeltaLogPaths().isEmpty() && !HoodieRealtimeInputFormatUtils.requiredProjectionFieldsExistInConf(jobConf));
}
/**
* Hive will append read columns' ids to old columns' ids during getRecordReader. In some cases, e.g. SELECT COUNT(*),
* the read columns' id is an empty string and Hive will combine it with Hoodie required projection ids and becomes