[HUDI-2177][HUDI-2200] Adding virtual keys support for MOR table (#3315)
This commit is contained in:
committed by
GitHub
parent
dde57b293c
commit
fe508376fa
@@ -18,10 +18,6 @@
|
||||
|
||||
package org.apache.hudi.table.action.cluster;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringGroup;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
@@ -36,6 +32,7 @@ import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.log.HoodieFileSliceReader;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
@@ -55,6 +52,11 @@ import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy;
|
||||
import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
@@ -98,7 +100,7 @@ public class SparkExecuteClusteringCommitActionExecutor<T extends HoodieRecordPa
|
||||
|
||||
JavaRDD<WriteStatus>[] writeStatuses = convertStreamToArray(writeStatusRDDStream);
|
||||
JavaRDD<WriteStatus> writeStatusRDD = engineContext.union(writeStatuses);
|
||||
|
||||
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> writeMetadata = buildWriteMetadata(writeStatusRDD);
|
||||
JavaRDD<WriteStatus> statuses = updateIndex(writeStatusRDD, writeMetadata);
|
||||
writeMetadata.setWriteStats(statuses.map(WriteStatus::getStat).collect());
|
||||
@@ -129,7 +131,7 @@ public class SparkExecuteClusteringCommitActionExecutor<T extends HoodieRecordPa
|
||||
/**
|
||||
* Validate actions taken by clustering. In the first implementation, we validate at least one new file is written.
|
||||
* But we can extend this to add more validation. E.g. number of records read = number of records written etc.
|
||||
*
|
||||
*
|
||||
* We can also make these validations in BaseCommitActionExecutor to reuse pre-commit hooks for multiple actions.
|
||||
*/
|
||||
private void validateWriteResult(HoodieWriteMetadata<JavaRDD<WriteStatus>> writeMetadata) {
|
||||
@@ -211,8 +213,11 @@ public class SparkExecuteClusteringCommitActionExecutor<T extends HoodieRecordPa
|
||||
.withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
|
||||
.build();
|
||||
|
||||
HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
|
||||
recordIterators.add(HoodieFileSliceReader.getFileSliceReader(baseFileReader, scanner, readerSchema,
|
||||
table.getMetaClient().getTableConfig().getPayloadClass()));
|
||||
tableConfig.getPayloadClass(),
|
||||
tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(),
|
||||
tableConfig.getPartitionFieldProp()))));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
|
||||
+ " and " + clusteringOp.getDeltaFilePaths(), e);
|
||||
|
||||
Reference in New Issue
Block a user