1
0

[HUDI-4097] add table info to jobStatus (#5529)

Co-authored-by: wqwl611 <wqwl611@gmail.com>
This commit is contained in:
wqwl611
2022-05-14 09:01:15 +08:00
committed by GitHub
parent 5c4813f101
commit 52e63b39d6
27 changed files with 41 additions and 41 deletions

View File

@@ -117,7 +117,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
@Override
public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata,
String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
context.setJobStatus(this.getClass().getSimpleName(), "Committing stats");
context.setJobStatus(this.getClass().getSimpleName(), "Committing stats: " + config.getTableName());
List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
}
@@ -303,7 +303,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
protected void completeCompaction(HoodieCommitMetadata metadata,
HoodieTable table,
String compactionCommitTime) {
this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction");
this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction: " + config.getTableName());
List<HoodieWriteStat> writeStats = metadata.getWriteStats();
final HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime);
try {

View File

@@ -126,7 +126,7 @@ public class SparkHoodieBloomIndexHelper extends BaseHoodieBloomIndexHelper {
if (config.getBloomIndexPruneByRanges()) {
// we will just try exploding the input and then count to determine comparisons
// FIX(vc): Only do sampling here and extrapolate?
context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files");
context.setJobStatus(this.getClass().getSimpleName(), "Compute all comparisons needed between records and files: " + config.getTableName());
fileToComparisons = fileComparisonsRDD.mapToPair(t -> t).countByKey();
} else {
fileToComparisons = new HashMap<>();

View File

@@ -334,7 +334,7 @@ public class SparkBootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>
})
.collect(Collectors.toList());
context.setJobStatus(this.getClass().getSimpleName(), "Bootstrap metadata table.");
context.setJobStatus(this.getClass().getSimpleName(), "Bootstrap metadata table: " + config.getTableName());
return context.parallelize(bootstrapPaths, config.getBootstrapParallelism())
.map(partitionFsPair -> getMetadataHandler(config, table, partitionFsPair.getRight().getRight()).runMetadataBootstrap(partitionFsPair.getLeft(),
partitionFsPair.getRight().getLeft(), keyGenerator));

View File

@@ -113,7 +113,7 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
}
private HoodieData<HoodieRecord<T>> clusteringHandleUpdate(HoodieData<HoodieRecord<T>> inputRecords, Set<HoodieFileGroupId> fileGroupsInPendingClustering) {
context.setJobStatus(this.getClass().getSimpleName(), "Handling updates which are under clustering");
context.setJobStatus(this.getClass().getSimpleName(), "Handling updates which are under clustering: " + config.getTableName());
UpdateStrategy<T, HoodieData<HoodieRecord<T>>> updateStrategy = (UpdateStrategy<T, HoodieData<HoodieRecord<T>>>) ReflectionUtils
.loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering);
Pair<HoodieData<HoodieRecord<T>>, Set<HoodieFileGroupId>> recordsAndPendingClusteringFileGroups =
@@ -152,7 +152,7 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
WorkloadProfile workloadProfile = null;
if (isWorkloadProfileNeeded()) {
context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile");
context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile: " + config.getTableName());
workloadProfile = new WorkloadProfile(buildProfile(inputRecords), operationType, table.getIndex().canIndexLogFiles());
LOG.info("Input workload profile :" + workloadProfile);
}
@@ -168,7 +168,7 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
table.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet());
HoodieData<HoodieRecord<T>> inputRecordsWithClusteringUpdate = fileGroupsInPendingClustering.isEmpty() ? inputRecords : clusteringHandleUpdate(inputRecords, fileGroupsInPendingClustering);
context.setJobStatus(this.getClass().getSimpleName(), "Doing partition and writing data");
context.setJobStatus(this.getClass().getSimpleName(), "Doing partition and writing data: " + config.getTableName());
HoodieData<WriteStatus> writeStatuses = mapPartitionsAsRDD(inputRecordsWithClusteringUpdate, partitioner);
HoodieWriteMetadata<HoodieData<WriteStatus>> result = new HoodieWriteMetadata<>();
updateIndexAndCommitIfNeeded(writeStatuses, result);
@@ -280,7 +280,7 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
@Override
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<HoodieData<WriteStatus>> result) {
context.setJobStatus(this.getClass().getSimpleName(), "Commit write status collect");
context.setJobStatus(this.getClass().getSimpleName(), "Commit write status collect: " + config.getTableName());
commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collectAsList());
}

View File

@@ -266,7 +266,7 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> extends SparkHo
}
if (partitionPaths != null && partitionPaths.size() > 0) {
context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions");
context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions: " + config.getTableName());
JavaRDD<String> partitionPathRdds = jsc.parallelize(partitionPaths, partitionPaths.size());
partitionSmallFilesMap = partitionPathRdds.mapToPair((PairFunction<String, String, List<SmallFile>>)
partitionPath -> new Tuple2<>(partitionPath, getSmallFiles(partitionPath))).collectAsMap();