You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/04/30 15:29:05 UTC

[GitHub] [incubator-hudi] vinothchandar commented on a change in pull request #1577: [WIP] [HUDI-855] Run Auto Cleaner in parallel with ingestion

vinothchandar commented on a change in pull request #1577:
URL: https://github.com/apache/incubator-hudi/pull/1577#discussion_r418092315



##########
File path: hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
##########
@@ -339,8 +352,13 @@ protected void postCommit(HoodieCommitMetadata metadata, String instantTime,
       archiveLog.archiveIfRequired(jsc);
       if (config.isAutoClean()) {
         // Call clean to cleanup if there is anything to cleanup after the commit,
-        LOG.info("Auto cleaning is enabled. Running cleaner now");
-        clean(instantTime);
+        if (config.isRunParallelAutoClean()) {

Review comment:
       let's move this into its own method for readability?

##########
File path: hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
##########
@@ -677,4 +714,27 @@ private void rollbackPendingCommits() {
     });
     return compactionInstantTimeOpt;
   }
+
+  /**
+   * Auto Clean service running concurrently.
+   */
+  private static class AutoCleanerService extends AbstractAsyncService {

Review comment:
       lets move this to its own class?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/async/AbstractAsyncService.java
##########
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.utilities.deltastreamer;
+package org.apache.hudi.common.async;

Review comment:
       does this really belong in hudi-common or it can stay in hudi-client (execution package?)

##########
File path: hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
##########
@@ -677,4 +714,27 @@ private void rollbackPendingCommits() {
     });
     return compactionInstantTimeOpt;
   }
+
+  /**
+   * Auto Clean service running concurrently.
+   */
+  private static class AutoCleanerService extends AbstractAsyncService {
+
+    private final HoodieWriteClient writeClient;
+    private final String cleanInstant;
+
+    private AutoCleanerService(HoodieWriteClient writeClient, String cleanInstant) {
+      this.writeClient = writeClient;
+      this.cleanInstant = cleanInstant;
+    }
+
+    @Override
+    protected Pair<CompletableFuture, ExecutorService> startService() {
+      ExecutorService executor = Executors.newFixedThreadPool(1);

Review comment:
       move to constructor and reuse for the duration of a HoodieWriteClient ?

##########
File path: hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java
##########
@@ -111,10 +111,15 @@ public CleanPlanner(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
    * @throws IOException when underlying file-system throws this exception
    */
   public List<String> getPartitionPathsToClean(Option<HoodieInstant> newInstantToRetain) throws IOException {
-    if (config.incrementalCleanerModeEnabled() && newInstantToRetain.isPresent()
+
+    if (!newInstantToRetain.isPresent() && (HoodieCleaningPolicy.KEEP_LATEST_COMMITS == config.getCleanerPolicy())) {

Review comment:
       this is from the other PR. 

##########
File path: hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
##########
@@ -677,4 +714,27 @@ private void rollbackPendingCommits() {
     });
     return compactionInstantTimeOpt;
   }
+
+  /**
+   * Auto Clean service running concurrently.
+   */
+  private static class AutoCleanerService extends AbstractAsyncService {

Review comment:
       And even the `spawnAutoCleanerIfEnabled` and `waitForAutoCleanerToShutdown` as static helpers in this class.. (trying to avoid code creep inside HoodieWriteClient :))




----------------------------------------------------------------
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.

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