[ https://issues.apache.org/jira/browse/HIVE-23324?focusedWorklogId=461918&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-461918 ]
ASF GitHub Bot logged work on HIVE-23324: ----------------------------------------- Author: ASF GitHub Bot Created on: 22/Jul/20 09:12 Start Date: 22/Jul/20 09:12 Worklog Time Spent: 10m Work Description: deniskuzZ commented on a change in pull request #1275: URL: https://github.com/apache/hive/pull/1275#discussion_r458651196 ########## 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: shutdown should be called here, otherwise you can terminate threads from 2nd iteration ---------------------------------------------------------------- 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: 461918) Time Spent: 4h 40m (was: 4.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: 4h 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)