codope commented on a change in pull request #3142: URL: https://github.com/apache/hudi/pull/3142#discussion_r661474972
########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java ########## @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.async; + +import org.apache.hudi.client.AbstractClusteringClient; +import org.apache.hudi.client.AbstractHoodieWriteClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.IntStream; + +public abstract class AsyncClusteringService extends HoodieAsyncService { + + private static final Logger LOG = LogManager.getLogger(AsyncClusteringService.class); + + private final int maxConcurrentClustering; + private transient AbstractClusteringClient clusteringClient; + private transient BlockingQueue<HoodieInstant> pendingClustering = new LinkedBlockingQueue<>(); + private transient ReentrantLock queueLock = new ReentrantLock(); + private transient Condition consumed = queueLock.newCondition(); + + public AsyncClusteringService(AbstractHoodieWriteClient writeClient) { + this(writeClient, false); + } + + public AsyncClusteringService(AbstractHoodieWriteClient writeClient, boolean runInDaemonMode) { + super(runInDaemonMode); + this.clusteringClient = createClusteringClient(writeClient); + this.maxConcurrentClustering = 1; + } + + protected abstract AbstractClusteringClient createClusteringClient(AbstractHoodieWriteClient client); + + public void enqueuePendingClustering(HoodieInstant instant) { + LOG.info("Enqueuing new pending clustering instant: " + instant.getTimestamp()); + pendingClustering.add(instant); + } + + public void waitTillPendingClusteringReducesTo(int numPendingClustering) throws InterruptedException { + try { + queueLock.lock(); + while (!isShutdown() && (pendingClustering.size() > numPendingClustering)) { + consumed.await(); + } + } finally { + queueLock.unlock(); + } + } + + private HoodieInstant fetchNextClusteringInstant() throws InterruptedException { + LOG.info("Waiting for next clustering instant for 10 seconds"); + HoodieInstant instant = pendingClustering.poll(10, TimeUnit.SECONDS); + if (instant != null) { + try { + queueLock.lock(); + // Signal waiting thread + consumed.signal(); + } finally { + queueLock.unlock(); + } + } + return instant; + } + + /** + * Start clustering service. + */ + @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; + }); + + return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentClustering).mapToObj(i -> CompletableFuture.supplyAsync(() -> { + try { + while (!isShutdownRequested()) { + final HoodieInstant instant = fetchNextClusteringInstant(); + if (null != instant) { + LOG.info("Starting clustering for instant " + instant); + clusteringClient.cluster(instant); + LOG.info("Finished clustering for instant " + instant); + } + } + LOG.info("Clustering executor shutting down properly"); + } catch (InterruptedException ie) { + LOG.warn("Clustering executor got interrupted exception! Stopping", ie); + } catch (IOException e) { + LOG.error("Clustering executor failed", e); + throw new HoodieIOException(e.getMessage(), e); + } + return true; + }, executor)).toArray(CompletableFuture[]::new)), executor); + } + + public synchronized void updateWriteClient(AbstractHoodieWriteClient writeClient) { Review comment: Moved some common logic to `HoodieAsyncService`. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
