[ 
https://issues.apache.org/jira/browse/HUDI-1483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17373799#comment-17373799
 ] 

ASF GitHub Bot commented on HUDI-1483:
--------------------------------------

nsivabalan commented on a change in pull request #3142:
URL: https://github.com/apache/hudi/pull/3142#discussion_r663254225



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.IntStream;
+
+/**
+ * Async clustering service that runs in a separate thread.
+ * Currently, only one clustering thread is allowed to run at any time.
+ */
+public abstract class AsyncClusteringService extends HoodieAsyncService {
+
+  private static final long serialVersionUID = 1L;
+  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();

Review comment:
       let's sync up on this. I feel we can take these vars also one level up 
and reuse across clustering and compaction. 

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java
##########
@@ -165,4 +169,49 @@ private void monitorThreads(Function<Boolean, Boolean> 
onShutdownCallback) {
   public boolean isRunInDaemonMode() {
     return runInDaemonMode;
   }
+
+  /**
+   * Wait till outstanding pending compaction/clustering reduces to the passed 
in value.
+   *
+   * @param numPending Maximum pending compactions/clustering allowed
+   * @param pendingInstants Currently enqueued pending compaction/clustering 
instants
+   * @param queueLock

Review comment:
       java docs on params as well :) 

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java
##########
@@ -296,6 +297,11 @@ public static void main(String[] args) throws IOException {
         + "outstanding compactions is less than this number")
     public Integer maxPendingCompactions = 5;
 
+    @Parameter(names = {"--max-pending-clustering"},

Review comment:
       out of curiosity. how come we have maxPendingCompaction/Clustering 
property defined as first class config for multiTableDeltaStreamer, but don't 
see the properties to enable/disable them. I assume those are fetched from 
property file for each source. So, why not fetch these properties also from the 
property file? I know this is not specific to clustering, but it was how 
compaction was defined. 

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java
##########
@@ -165,4 +169,49 @@ private void monitorThreads(Function<Boolean, Boolean> 
onShutdownCallback) {
   public boolean isRunInDaemonMode() {
     return runInDaemonMode;
   }
+
+  /**
+   * Wait till outstanding pending compaction/clustering reduces to the passed 
in value.
+   *
+   * @param numPending Maximum pending compactions/clustering allowed
+   * @param pendingInstants Currently enqueued pending compaction/clustering 
instants
+   * @param queueLock
+   * @param consumed
+   * @throws InterruptedException
+   */
+  public void waitTillPendingActionReducesTo(int numPending, 
BlockingQueue<HoodieInstant> pendingInstants,
+                                             ReentrantLock queueLock, 
Condition consumed) throws InterruptedException {
+    try {
+      queueLock.lock();
+      while (!isShutdown() && (pendingInstants.size() > numPending)) {
+        consumed.await();
+      }
+    } finally {
+      queueLock.unlock();
+    }
+  }
+
+  /**
+   * Fetch Next pending compaction/clustering instant if available.
+   *
+   * @param pendingInstants Currently enqueued pending compaction/clustering 
instants
+   * @param queueLock
+   * @param consumed

Review comment:
       same here. also "return" as well

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
##########
@@ -616,6 +651,7 @@ public DeltaSync getDeltaSync() {
           }
         } finally {
           shutdownCompactor(error);
+          shutdownClusteringService(error);

Review comment:
       can we have a single method and call it shutdownAsyncServices or 
backgroundServices and shut down all such services within that method ?

##########
File path: 
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
##########
@@ -86,6 +93,11 @@ class HoodieStreamingSink(sqlContext: SQLContext,
             asyncCompactorService.enqueuePendingCompaction(
               new HoodieInstant(State.REQUESTED, 
HoodieTimeline.COMPACTION_ACTION, compactionInstantOps.get()))
           }
+          if (clusteringInstant.isPresent) {
+            asyncClusteringService.enqueuePendingClustering(new HoodieInstant(
+              State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, 
clusteringInstant.get()
+            ))
+          }

Review comment:
       I know we synced up f2f on this. but for book keeping purposes, can you 
leave a comment as if you haven't addressed my feedback. 

##########
File path: 
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java
##########
@@ -150,6 +152,9 @@ public static HoodieWriteConfig createHoodieConfig(String 
schemaStr, String base
         .withCompactionConfig(HoodieCompactionConfig.newBuilder()
             
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY().key()))
             .withInlineCompaction(inlineCompact).build())
+        .withClusteringConfig(HoodieClusteringConfig.newBuilder()
+            .withInlineClustering(!asyncClusteringEnabled)

Review comment:
       isn't this supposed to be 
INLINE_CLUSTERING_PROP(hoodie.clustering.inline)? what happens if there is some 
contradiction among this existing property and the new 
property(hoodie.datasource.clustering.async.enable)?
   




-- 
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: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> async clustering for deltastreamer
> ----------------------------------
>
>                 Key: HUDI-1483
>                 URL: https://issues.apache.org/jira/browse/HUDI-1483
>             Project: Apache Hudi
>          Issue Type: Sub-task
>            Reporter: liwei
>            Assignee: liwei
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 0.9.0
>
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to