1
0

[HUDI-2849] Improve SparkUI job description for write path (#4222)

This commit is contained in:
Yuwei XIAO
2021-12-10 23:22:37 +08:00
committed by GitHub
parent c7473a7b0c
commit f194566ed4
9 changed files with 12 additions and 2 deletions

View File

@@ -265,6 +265,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
* @param metadata instance of {@link HoodieCommitMetadata}.
*/
protected void writeTableMetadata(HoodieTable table, String instantTime, String actionType, HoodieCommitMetadata metadata) {
context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table");
table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, instantTime,
table.isTableServiceAction(actionType)));
}

View File

@@ -464,6 +464,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
// List all partitions in the basePath of the containing dataset
LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath());
engineContext.setJobStatus(this.getClass().getSimpleName(), "Bootstrap: initializing metadata table by listing files and partitions");
List<DirectoryInfo> dirInfoList = listAllPartitions(dataMetaClient);
// During bootstrap, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out of these

View File

@@ -72,6 +72,7 @@ public class CleanPlanActionExecutor<T extends HoodieRecordPayload, I, K, O> ext
try {
CleanPlanner<T, I, K, O> planner = new CleanPlanner<>(context, table, config);
Option<HoodieInstant> earliestInstant = planner.getEarliestCommitToRetain();
context.setJobStatus(this.getClass().getSimpleName(), "Obtaining list of partitions to be cleaned");
List<String> partitionsToClean = planner.getPartitionPathsToClean(earliestInstant);
if (partitionsToClean.isEmpty()) {
@@ -82,7 +83,7 @@ public class CleanPlanActionExecutor<T extends HoodieRecordPayload, I, K, O> ext
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
LOG.info("Using cleanerParallelism: " + cleanerParallelism);
context.setJobStatus(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned");
context.setJobStatus(this.getClass().getSimpleName(), "Generating list of file slices to be cleaned");
Map<String, List<HoodieCleanFileInfo>> cleanOps = context
.map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism)

View File

@@ -48,6 +48,7 @@ public abstract class AbstractWriteHelper<T extends HoodieRecordPayload, I, K, O
I taggedRecords = dedupedRecords;
if (performTagging) {
// perform index loop up to get existing location of records
context.setJobStatus(this.getClass().getSimpleName(), "Tagging");
taggedRecords = tag(dedupedRecords, context, table);
}
Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now());

View File

@@ -118,6 +118,7 @@ public class ScheduleCompactionActionExecutor<T extends HoodieRecordPayload, I,
.collect(Collectors.toSet());
// exclude files in pending clustering from compaction.
fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan");
return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
} catch (IOException e) {
throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);

View File

@@ -85,6 +85,7 @@ public abstract class WriteMarkers implements Serializable {
*/
public void quietDeleteMarkerDir(HoodieEngineContext context, int parallelism) {
try {
context.setJobStatus(this.getClass().getSimpleName(), "Deleting marker directory");
deleteMarkerDir(context, parallelism);
} catch (HoodieIOException ioe) {
LOG.warn("Error deleting marker directory for instant " + instantTime, ioe);