[HUDI-3485] Adding scheduler pool configs for async clustering (#5043)
This commit is contained in:
committed by
GitHub
parent
5c1b482a1b
commit
4fed8dd319
@@ -21,6 +21,8 @@ package org.apache.hudi.async;
|
||||
|
||||
import org.apache.hudi.client.BaseClusterer;
|
||||
import org.apache.hudi.client.BaseHoodieWriteClient;
|
||||
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.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
@@ -42,18 +44,21 @@ public abstract class AsyncClusteringService extends HoodieAsyncTableService {
|
||||
|
||||
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;
|
||||
|
||||
public AsyncClusteringService(BaseHoodieWriteClient writeClient) {
|
||||
this(writeClient, false);
|
||||
public AsyncClusteringService(HoodieEngineContext context, BaseHoodieWriteClient writeClient) {
|
||||
this(context, writeClient, false);
|
||||
}
|
||||
|
||||
public AsyncClusteringService(BaseHoodieWriteClient writeClient, boolean runInDaemonMode) {
|
||||
public AsyncClusteringService(HoodieEngineContext context, BaseHoodieWriteClient writeClient, boolean runInDaemonMode) {
|
||||
super(writeClient.getConfig(), runInDaemonMode);
|
||||
this.clusteringClient = createClusteringClient(writeClient);
|
||||
this.maxConcurrentClustering = 1;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
protected abstract BaseClusterer createClusteringClient(BaseHoodieWriteClient client);
|
||||
@@ -72,6 +77,9 @@ public abstract class AsyncClusteringService extends HoodieAsyncTableService {
|
||||
|
||||
return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentClustering).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
|
||||
try {
|
||||
// Set Compactor Pool Name for allowing users to prioritize compaction
|
||||
LOG.info("Setting pool name for clustering to " + CLUSTERING_POOL_NAME);
|
||||
context.setProperty(EngineProperty.CLUSTERING_POOL_NAME, CLUSTERING_POOL_NAME);
|
||||
while (!isShutdownRequested()) {
|
||||
final HoodieInstant instant = fetchNextAsyncServiceInstant();
|
||||
if (null != instant) {
|
||||
|
||||
Reference in New Issue
Block a user