vinothchandar commented on a change in pull request #4252:
URL: https://github.com/apache/hudi/pull/4252#discussion_r766201489



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -423,7 +423,11 @@ protected void preWrite(String instantTime, 
WriteOperationType writeOperationTyp
       HoodieTableMetaClient metaClient) {
     setOperationType(writeOperationType);
     this.lastCompletedTxnAndMetadata = 
TransactionUtils.getLastCompletedTxnInstantAndMetadata(metaClient);
-    this.asyncCleanerService = 
AsyncCleanerService.startAsyncCleaningIfEnabled(this);
+    if (null == this.asyncCleanerService) {

Review comment:
       this changes behavior? currently the async cleaner is started for each 
write, seems like we are reusing this now? Is this intentional. Can we retain 
existing behavior?

##########
File path: 
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java
##########
@@ -281,7 +281,11 @@ public void initMetadataWriter() {
    * checkpoint finish.
    */
   public void startAsyncCleaning() {
-    this.asyncCleanerService = 
AsyncCleanerService.startAsyncCleaningIfEnabled(this);
+    if (this.asyncCleanerService == null) {

Review comment:
       some way to simplify this if-else into a single line. Can we use option 
instead of null?

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java
##########
@@ -141,35 +140,17 @@ public void start(Function<Boolean, Boolean> 
onShutdownCallback) {
   protected abstract Pair<CompletableFuture, ExecutorService> startService();
 
   /**
-   * A monitor thread is started which would trigger a callback if the service 
is shutdown.
+   * Add shutdown callback for the completable future.
    * 
-   * @param onShutdownCallback
+   * @param callback The callback
    */
-  private void monitorThreads(Function<Boolean, Boolean> onShutdownCallback) {
-    LOG.info("Submitting monitor thread !!");
-    Executors.newSingleThreadExecutor(r -> {
-      Thread t = new Thread(r, "Monitor Thread");
-      t.setDaemon(isRunInDaemonMode());

Review comment:
       yes this seems okay

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java
##########
@@ -37,28 +37,26 @@
   private static final Logger LOG = 
LogManager.getLogger(AsyncCleanerService.class);
 
   private final AbstractHoodieWriteClient writeClient;
-  private final String cleanInstantTime;
   private final transient ExecutorService executor = 
Executors.newSingleThreadExecutor();
 
-  protected AsyncCleanerService(AbstractHoodieWriteClient writeClient, String 
cleanInstantTime) {

Review comment:
       is this change necessary to fix the leak?  




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


Reply via email to