1
0

[HUDI-1437] support more accurate spark JobGroup for better performance tracking (#2322)

This commit is contained in:
lw0090
2020-12-18 07:20:13 +08:00
committed by GitHub
parent 14d5d1100c
commit 8b5d6f9430
14 changed files with 23 additions and 9 deletions

View File

@@ -53,7 +53,7 @@ public abstract class AsyncCompactService extends HoodieAsyncService {
private final int maxConcurrentCompaction;
private transient AbstractCompactor compactor;
private transient HoodieEngineContext context;
protected transient HoodieEngineContext context;
private transient BlockingQueue<HoodieInstant> pendingCompactions = new LinkedBlockingQueue<>();
private transient ReentrantLock queueLock = new ReentrantLock();
private transient Condition consumed = queueLock.newCondition();

View File

@@ -79,6 +79,7 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
@Override
public void close() {
stopEmbeddedServerView(true);
this.context.setJobStatus("", "");
}
private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) {

View File

@@ -674,8 +674,7 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
*/
protected abstract void completeCompaction(HoodieCommitMetadata metadata, O writeStatuses,
HoodieTable<T, I, K, O> table, String compactionCommitTime);
/**
* Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file to the .requested file
*

View File

@@ -68,7 +68,7 @@ public class ReplaceArchivalHelper implements Serializable {
public static boolean deleteReplacedFileGroups(HoodieEngineContext context, HoodieTableMetaClient metaClient,
TableFileSystemView fileSystemView,
HoodieInstant instant, List<String> replacedPartitions) {
context.setJobStatus(ReplaceArchivalHelper.class.getSimpleName(), "Delete replaced file groups");
List<Boolean> f = context.map(replacedPartitions, partition -> {
Stream<FileSlice> fileSlices = fileSystemView.getReplacedFileGroupsBeforeOrOn(instant.getTimestamp(), partition)
.flatMap(HoodieFileGroup::getAllRawFileSlices);

View File

@@ -403,6 +403,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.map(new ArrayList<>(invalidFilesByPartition.values()), partitionWithFileList -> {
final FileSystem fileSystem = metaClient.getFs();
LOG.info("Deleting invalid data files=" + partitionWithFileList);

View File

@@ -135,6 +135,7 @@ public class MarkerFiles implements Serializable {
if (subDirectories.size() > 0) {
parallelism = Math.min(subDirectories.size(), parallelism);
SerializableConfiguration serializedConf = new SerializableConfiguration(fs.getConf());
context.setJobStatus(this.getClass().getSimpleName(), "Obtaining marker files for all created, merged paths");
dataFiles.addAll(context.flatMap(subDirectories, directory -> {
Path path = new Path(directory);
FileSystem fileSystem = path.getFileSystem(serializedConf.get());