[ https://issues.apache.org/jira/browse/HIVE-23324?focusedWorklogId=461940&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-461940 ]
ASF GitHub Bot logged work on HIVE-23324: ----------------------------------------- Author: ASF GitHub Bot Created on: 22/Jul/20 09:55 Start Date: 22/Jul/20 09:55 Worklog Time Spent: 10m Work Description: adesh-rao commented on a change in pull request #1275: URL: https://github.com/apache/hive/pull/1275#discussion_r458675972 ########## File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java ########## @@ -149,17 +156,25 @@ public void run() { String runAs = resolveUserToRunAs(tblNameOwners, t, p); /* checkForCompaction includes many file metadata checks and may be expensive. * Therefore, using a thread pool here and running checkForCompactions in parallel */ - compactionList.add(CompletableFuture.runAsync(ThrowingRunnable.unchecked(() -> - scheduleCompactionIfRequired(ci, t, p, runAs)), compactionExecutor)); + completionService.submit(() -> { + ThrowingRunnable.unchecked(() -> scheduleCompactionIfRequired(ci, t, p, runAs)); + return null; Review comment: Same as above. Removed it. ########## File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java ########## @@ -80,39 +98,58 @@ public void run() { HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, TimeUnit.MILLISECONDS); } - do { - TxnStore.MutexAPI.LockHandle handle = null; - long startedAt = -1; - // Make sure nothing escapes this run method and kills the metastore at large, - // so wrap it in a big catch Throwable statement. - try { - handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.Cleaner.name()); - startedAt = System.currentTimeMillis(); - long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner(); - for(CompactionInfo compactionInfo : txnHandler.findReadyToClean()) { - clean(compactionInfo, minOpenTxnId); + try { + do { + TxnStore.MutexAPI.LockHandle handle = null; + long startedAt = -1; + // Make sure nothing escapes this run method and kills the metastore at large, + // so wrap it in a big catch Throwable statement. + try { + handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.Cleaner.name()); + startedAt = System.currentTimeMillis(); + long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner(); + int count = 0; + for(CompactionInfo compactionInfo : txnHandler.findReadyToClean()) { + completionService.submit(() -> { + ThrowingRunnable.unchecked(() -> clean(compactionInfo, minOpenTxnId)); + return null; + }); + count++; + } + + for(int i=0; i<count; i++) { + try { + completionService.take().get(); Review comment: Agree, changed it. ########## File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java ########## @@ -80,39 +98,58 @@ public void run() { HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL, TimeUnit.MILLISECONDS); } - do { - TxnStore.MutexAPI.LockHandle handle = null; - long startedAt = -1; - // Make sure nothing escapes this run method and kills the metastore at large, - // so wrap it in a big catch Throwable statement. - try { - handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.Cleaner.name()); - startedAt = System.currentTimeMillis(); - long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner(); - for(CompactionInfo compactionInfo : txnHandler.findReadyToClean()) { - clean(compactionInfo, minOpenTxnId); + try { + do { + TxnStore.MutexAPI.LockHandle handle = null; + long startedAt = -1; + // Make sure nothing escapes this run method and kills the metastore at large, + // so wrap it in a big catch Throwable statement. + try { + handle = txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.Cleaner.name()); + startedAt = System.currentTimeMillis(); + long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner(); + int count = 0; + for(CompactionInfo compactionInfo : txnHandler.findReadyToClean()) { + completionService.submit(() -> { + ThrowingRunnable.unchecked(() -> clean(compactionInfo, minOpenTxnId)); + return null; + }); + count++; + } + + for(int i=0; i<count; i++) { + try { + completionService.take().get(); + } catch (InterruptedException| ExecutionException ignore) { + // What should we do here? + } + } + } catch (Throwable t) { + LOG.error("Caught an exception in the main loop of compactor cleaner, " + + StringUtils.stringifyException(t)); } - } catch (Throwable t) { - LOG.error("Caught an exception in the main loop of compactor cleaner, " + - StringUtils.stringifyException(t)); - } - finally { - if (handle != null) { - handle.releaseLocks(); + finally { + if (handle != null) { Review comment: Fixed this. ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 461940) Time Spent: 5h 40m (was: 5.5h) > Parallelise compaction directory cleaning process > ------------------------------------------------- > > Key: HIVE-23324 > URL: https://issues.apache.org/jira/browse/HIVE-23324 > Project: Hive > Issue Type: Improvement > Reporter: Marton Bod > Assignee: Adesh Kumar Rao > Priority: Major > Labels: pull-request-available > Time Spent: 5h 40m > Remaining Estimate: 0h > > Initiator processes the various compaction candidates in parallel, so we > could follow a similar approach in Cleaner where we currently clean the > directories sequentially. -- This message was sent by Atlassian Jira (v8.3.4#803005)