[HUDI-4097] add table info to jobStatus (#5529)
Co-authored-by: wqwl611 <wqwl611@gmail.com>
This commit is contained in:
@@ -334,7 +334,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
* @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");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Committing to metadata table: " + config.getTableName());
|
||||
table.getMetadataWriter(instantTime).ifPresent(w -> ((HoodieTableMetadataWriter) w).update(metadata, instantTime,
|
||||
table.isTableServiceAction(actionType)));
|
||||
}
|
||||
@@ -1038,7 +1038,7 @@ public abstract class BaseHoodieWriteClient<T extends HoodieRecordPayload, I, K,
|
||||
HoodieInstant ownerInstant = new HoodieInstant(true, HoodieTimeline.INDEXING_ACTION, dropInstant);
|
||||
this.txnManager.beginTransaction(Option.of(ownerInstant), Option.empty());
|
||||
try {
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Dropping partitions from metadata table");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Dropping partitions from metadata table: " + config.getTableName());
|
||||
table.getMetadataWriter(dropInstant).ifPresent(w -> {
|
||||
try {
|
||||
((HoodieTableMetadataWriter) w).dropMetadataPartitions(partitionTypes);
|
||||
|
||||
@@ -85,7 +85,7 @@ public class CompactionAdminClient extends BaseHoodieClient {
|
||||
if (plan.getOperations() != null) {
|
||||
List<CompactionOperation> ops = plan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Validate compaction operations");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Validate compaction operations: " + config.getTableName());
|
||||
return context.map(ops, op -> {
|
||||
try {
|
||||
return validateCompactionOperation(metaClient, compactionInstant, op, Option.of(fsView));
|
||||
@@ -351,7 +351,7 @@ public class CompactionAdminClient extends BaseHoodieClient {
|
||||
} else {
|
||||
LOG.info("The following compaction renaming operations needs to be performed to un-schedule");
|
||||
if (!dryRun) {
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Execute unschedule operations");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Execute unschedule operations: " + config.getTableName());
|
||||
return context.map(renameActions, lfPair -> {
|
||||
try {
|
||||
LOG.info("RENAME " + lfPair.getLeft().getPath() + " => " + lfPair.getRight().getPath());
|
||||
@@ -394,7 +394,7 @@ public class CompactionAdminClient extends BaseHoodieClient {
|
||||
"Number of Compaction Operations :" + plan.getOperations().size() + " for instant :" + compactionInstant);
|
||||
List<CompactionOperation> ops = plan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Generate compaction unscheduling operations");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Generate compaction unscheduling operations: " + config.getTableName());
|
||||
return context.flatMap(ops, op -> {
|
||||
try {
|
||||
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op,
|
||||
|
||||
@@ -519,7 +519,7 @@ public class HoodieTimelineArchiver<T extends HoodieAvroPayload, I, K, O> {
|
||||
new Path(metaClient.getMetaPath(), archivedInstant.getFileName())
|
||||
).map(Path::toString).collect(Collectors.toList());
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants: " + config.getTableName());
|
||||
Map<String, Boolean> resultDeleteInstantFiles = deleteFilesParallelize(metaClient, instantFiles, context, false);
|
||||
|
||||
for (Map.Entry<String, Boolean> result : resultDeleteInstantFiles.entrySet()) {
|
||||
|
||||
@@ -83,7 +83,7 @@ public class HoodieIndexUtils {
|
||||
public static List<Pair<String, HoodieBaseFile>> getLatestBaseFilesForAllPartitions(final List<String> partitions,
|
||||
final HoodieEngineContext context,
|
||||
final HoodieTable hoodieTable) {
|
||||
context.setJobStatus(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions");
|
||||
context.setJobStatus(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions: " + hoodieTable.getConfig().getTableName());
|
||||
return context.flatMap(partitions, partitionPath -> {
|
||||
List<Pair<String, HoodieBaseFile>> filteredFiles =
|
||||
getLatestBaseFilesForPartition(partitionPath, hoodieTable).stream()
|
||||
|
||||
@@ -167,7 +167,7 @@ public class HoodieBloomIndex extends HoodieIndex<Object, Object> {
|
||||
.map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId()))
|
||||
.collect(toList());
|
||||
|
||||
context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)");
|
||||
context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on): " + config.getTableName());
|
||||
return context.map(partitionPathFileIDList, pf -> {
|
||||
try {
|
||||
HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf);
|
||||
@@ -209,7 +209,7 @@ public class HoodieBloomIndex extends HoodieIndex<Object, Object> {
|
||||
protected List<Pair<String, BloomIndexFileInfo>> loadColumnRangesFromMetaIndex(
|
||||
List<String> partitions, final HoodieEngineContext context, final HoodieTable<?, ?, ?, ?> hoodieTable) {
|
||||
// also obtain file ranges, if range pruning is enabled
|
||||
context.setJobStatus(this.getClass().getName(), "Load meta index key ranges for file slices");
|
||||
context.setJobStatus(this.getClass().getName(), "Load meta index key ranges for file slices: " + config.getTableName());
|
||||
|
||||
final String keyField = hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp();
|
||||
return context.flatMap(partitions, partitionName -> {
|
||||
|
||||
@@ -1047,7 +1047,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
||||
private void initialCommit(String createInstantTime, List<MetadataPartitionType> partitionTypes) {
|
||||
// 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(), "Initializing metadata table by listing files and partitions");
|
||||
engineContext.setJobStatus(this.getClass().getSimpleName(), "Initializing metadata table by listing files and partitions: " + dataWriteConfig.getTableName());
|
||||
|
||||
Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionToRecordsMap = new HashMap<>();
|
||||
|
||||
|
||||
@@ -566,7 +566,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
|
||||
private void deleteInvalidFilesByPartitions(HoodieEngineContext context, Map<String, List<Pair<String, String>>> invalidFilesByPartition) {
|
||||
// Now delete partially written files
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Delete invalid files generated during the write operation");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Delete invalid files generated during the write operation: " + config.getTableName());
|
||||
context.map(new ArrayList<>(invalidFilesByPartition.values()), partitionWithFileList -> {
|
||||
final FileSystem fileSystem = metaClient.getFs();
|
||||
LOG.info("Deleting invalid data files=" + partitionWithFileList);
|
||||
@@ -642,7 +642,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
}
|
||||
|
||||
// Now delete partially written files
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Delete all partially written files");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Delete all partially written files: " + config.getTableName());
|
||||
deleteInvalidFilesByPartitions(context, invalidPathsByPartition);
|
||||
|
||||
// Now ensure the deleted files disappear
|
||||
@@ -665,7 +665,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
*/
|
||||
private void waitForAllFiles(HoodieEngineContext context, Map<String, List<Pair<String, String>>> groupByPartition, FileVisibility visibility) {
|
||||
// This will either ensure all files to be deleted are present.
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Wait for all files to appear/disappear");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Wait for all files to appear/disappear: " + config.getTableName());
|
||||
boolean checkPassed =
|
||||
context.map(new ArrayList<>(groupByPartition.entrySet()), partitionWithFileList -> waitForCondition(partitionWithFileList.getKey(),
|
||||
partitionWithFileList.getValue().stream(), visibility), config.getFinalizeWriteParallelism())
|
||||
|
||||
@@ -132,7 +132,7 @@ public class CleanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends
|
||||
config.getCleanerParallelism());
|
||||
LOG.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions: " + config.getTableName());
|
||||
|
||||
Stream<Pair<String, CleanFileInfo>> filesToBeDeletedPerPartition =
|
||||
cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream()
|
||||
|
||||
@@ -96,7 +96,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");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Obtaining list of partitions to be cleaned: " + config.getTableName());
|
||||
List<String> partitionsToClean = planner.getPartitionPathsToClean(earliestInstant);
|
||||
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
@@ -107,7 +107,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(), "Generating list of file slices to be cleaned");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Generating list of file slices to be cleaned: " + config.getTableName());
|
||||
|
||||
Map<String, Pair<Boolean, List<CleanFileInfo>>> cleanOpsWithPartitionMeta = context
|
||||
.map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism)
|
||||
|
||||
@@ -49,7 +49,7 @@ public abstract class BaseWriteHelper<T extends HoodieRecordPayload, I, K, O, R>
|
||||
I taggedRecords = dedupedRecords;
|
||||
if (table.getIndex().requiresTagging(operationType)) {
|
||||
// perform index loop up to get existing location of records
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Tagging");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Tagging: " + table.getConfig().getTableName());
|
||||
taggedRecords = tag(dedupedRecords, context, table);
|
||||
}
|
||||
Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now());
|
||||
|
||||
@@ -133,7 +133,7 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
LOG.info("Compactor compacting " + operations + " files");
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices: " + config.getTableName());
|
||||
TaskContextSupplier taskContextSupplier = table.getTaskContextSupplier();
|
||||
return context.parallelize(operations).map(operation -> compact(
|
||||
compactionHandler, metaClient, config, operation, compactionInstantTime, taskContextSupplier))
|
||||
@@ -288,7 +288,7 @@ public abstract class HoodieCompactor<T extends HoodieRecordPayload, I, K, O> im
|
||||
|
||||
SliceView fileSystemView = hoodieTable.getSliceView();
|
||||
LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact: " + config.getTableName());
|
||||
|
||||
List<HoodieCompactionOperation> operations = context.flatMap(partitionPaths, partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
|
||||
@@ -88,7 +88,7 @@ public class RunCompactionActionExecutor<T extends HoodieRecordPayload> extends
|
||||
context, compactionPlan, table, configCopy, instantTime, compactionHandler);
|
||||
|
||||
compactor.maybePersist(statuses, config);
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Preparing compaction metadata");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Preparing compaction metadata: " + config.getTableName());
|
||||
List<HoodieWriteStat> updateStatusMap = statuses.map(WriteStatus::getStat).collectAsList();
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
|
||||
for (HoodieWriteStat stat : updateStatusMap) {
|
||||
|
||||
@@ -119,7 +119,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");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Compaction: generating compaction plan: " + config.getTableName());
|
||||
return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
|
||||
|
||||
@@ -72,7 +72,7 @@ public class BaseRollbackHelper implements Serializable {
|
||||
public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback,
|
||||
List<HoodieRollbackRequest> rollbackRequests) {
|
||||
int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions: " + config.getTableName());
|
||||
// If not for conversion to HoodieRollbackInternalRequests, code fails. Using avro model (HoodieRollbackRequest) within spark.parallelize
|
||||
// is failing with com.esotericsoftware.kryo.KryoException
|
||||
// stack trace: https://gist.github.com/nsivabalan/b6359e7d5038484f8043506c8bc9e1c8
|
||||
@@ -88,7 +88,7 @@ public class BaseRollbackHelper implements Serializable {
|
||||
public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback,
|
||||
List<HoodieRollbackRequest> rollbackRequests) {
|
||||
int parallelism = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade: " + config.getTableName());
|
||||
// If not for conversion to HoodieRollbackInternalRequests, code fails. Using avro model (HoodieRollbackRequest) within spark.parallelize
|
||||
// is failing with com.esotericsoftware.kryo.KryoException
|
||||
// stack trace: https://gist.github.com/nsivabalan/b6359e7d5038484f8043506c8bc9e1c8
|
||||
|
||||
@@ -88,7 +88,7 @@ public class ListingBasedRollbackStrategy implements BaseRollbackPlanActionExecu
|
||||
FSUtils.getAllPartitionPaths(context, table.getMetaClient().getBasePath(), false, false);
|
||||
int numPartitions = Math.max(Math.min(partitionPaths.size(), config.getRollbackParallelism()), 1);
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Creating Listing Rollback Plan");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Creating Listing Rollback Plan: " + config.getTableName());
|
||||
|
||||
HoodieTableType tableType = table.getMetaClient().getTableType();
|
||||
String baseFileExtension = getBaseFileExtension(metaClient);
|
||||
|
||||
@@ -84,7 +84,7 @@ public class SavepointActionExecutor<T extends HoodieRecordPayload, I, K, O> ext
|
||||
ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.GREATER_THAN_OR_EQUALS, lastCommitRetained),
|
||||
"Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained);
|
||||
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime);
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime + " " + table.getConfig().getTableName());
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath());
|
||||
Map<String, List<String>> latestFilesMap = context.mapToPair(partitions, partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
|
||||
@@ -84,7 +84,7 @@ public abstract class WriteMarkers implements Serializable {
|
||||
*/
|
||||
public void quietDeleteMarkerDir(HoodieEngineContext context, int parallelism) {
|
||||
try {
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Deleting marker directory");
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Deleting marker directory: " + basePath);
|
||||
deleteMarkerDir(context, parallelism);
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Error deleting marker directory for instant " + instantTime, e);
|
||||
|
||||
Reference in New Issue
Block a user