Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-4086] Use CustomizedThreadFactory in async compaction and clustering #5563

Merged
merged 1 commit into from
May 29, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.hudi.common.engine.EngineProperty;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.CustomizedThreadFactory;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;

Expand All @@ -42,13 +43,12 @@
*/
public abstract class AsyncClusteringService extends HoodieAsyncTableService {

public static final String CLUSTERING_POOL_NAME = "hoodiecluster";
private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(AsyncClusteringService.class);
public static final String CLUSTERING_POOL_NAME = "hoodiecluster";

private final int maxConcurrentClustering;
private transient BaseClusterer clusteringClient;
protected transient HoodieEngineContext context;
private transient BaseClusterer clusteringClient;

public AsyncClusteringService(HoodieEngineContext context, BaseHoodieWriteClient writeClient) {
this(context, writeClient, false);
Expand All @@ -69,12 +69,7 @@ public AsyncClusteringService(HoodieEngineContext context, BaseHoodieWriteClient
@Override
protected Pair<CompletableFuture, ExecutorService> startService() {
ExecutorService executor = Executors.newFixedThreadPool(maxConcurrentClustering,
r -> {
Thread t = new Thread(r, "async_clustering_thread");
t.setDaemon(isRunInDaemonMode());
return t;
});

new CustomizedThreadFactory("async_clustering_thread", isRunInDaemonMode()));
return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentClustering).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
try {
// Set Compactor Pool Name for allowing users to prioritize compaction
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.hudi.common.engine.EngineProperty;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.CustomizedThreadFactory;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;

Expand All @@ -39,17 +40,15 @@
*/
public abstract class AsyncCompactService extends HoodieAsyncTableService {

private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(AsyncCompactService.class);

/**
* This is the job pool used by async compaction.
*/
public static final String COMPACT_POOL_NAME = "hoodiecompact";

private static final long serialVersionUID = 1L;
private static final Logger LOG = LogManager.getLogger(AsyncCompactService.class);
private final int maxConcurrentCompaction;
private transient BaseCompactor compactor;
protected transient HoodieEngineContext context;
private transient BaseCompactor compactor;

public AsyncCompactService(HoodieEngineContext context, BaseHoodieWriteClient client) {
this(context, client, false);
Expand All @@ -70,11 +69,7 @@ public AsyncCompactService(HoodieEngineContext context, BaseHoodieWriteClient cl
@Override
protected Pair<CompletableFuture, ExecutorService> startService() {
ExecutorService executor = Executors.newFixedThreadPool(maxConcurrentCompaction,
r -> {
Thread t = new Thread(r, "async_compact_thread");
t.setDaemon(isRunInDaemonMode());
return t;
});
new CustomizedThreadFactory("async_compact_thread", isRunInDaemonMode()));
return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
try {
// Set Compactor Pool Name for allowing users to prioritize compaction
Expand Down Expand Up @@ -107,9 +102,9 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
}, executor)).toArray(CompletableFuture[]::new)), executor);
}


/**
* Check whether compactor thread needs to be stopped.
*
* @return
*/
protected boolean shouldStopCompactor() {
Expand Down