adesh-rao commented on a change in pull request #1275:
URL: https://github.com/apache/hive/pull/1275#discussion_r462133018
##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -66,53 +69,60 @@
private long cleanerCheckInterval = 0;
private ReplChangeManager replChangeManager;
+ private ExecutorService cleanerExecutor;
@Override
public void init(AtomicBoolean stop) throws Exception {
super.init(stop);
replChangeManager = ReplChangeManager.getInstance(conf);
+ cleanerCheckInterval = conf.getTimeVar(
+ HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RUN_INTERVAL,
TimeUnit.MILLISECONDS);
+ cleanerExecutor = CompactorUtil.createExecutorWithThreadFactory(
+
conf.getIntVar(HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_REQUEST_QUEUE),
+ COMPACTOR_CLEANER_THREAD_NAME_FORMAT);
}
@Override
public void run() {
- if (cleanerCheckInterval == 0) {
- cleanerCheckInterval = conf.getTimeVar(
- 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);
- }
- } catch (Throwable t) {
- LOG.error("Caught an exception in the main loop of compactor cleaner,
" +
- StringUtils.stringifyException(t));
- }
- finally {
- if (handle != null) {
- handle.releaseLocks();
- }
- }
- // Now, go back to bed until it's time to do this again
- long elapsedTime = System.currentTimeMillis() - startedAt;
- if (elapsedTime >= cleanerCheckInterval || stop.get()) {
- continue;
- } else {
+ 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 {
- Thread.sleep(cleanerCheckInterval - elapsedTime);
- } catch (InterruptedException ie) {
- // What can I do about it?
+ handle =
txnHandler.getMutexAPI().acquireLock(TxnStore.MUTEX_KEY.Cleaner.name());
+ startedAt = System.currentTimeMillis();
+ long minOpenTxnId = txnHandler.findMinOpenTxnIdForCleaner();
+ List<CompletableFuture> cleanerList = new ArrayList<>();
+ for(CompactionInfo compactionInfo : txnHandler.findReadyToClean()) {
+
cleanerList.add(CompletableFuture.runAsync(CompactorUtil.ThrowingRunnable.unchecked(()
->
+ clean(compactionInfo, minOpenTxnId)), cleanerExecutor));
+ }
+ CompletableFuture.allOf(cleanerList.toArray(new
CompletableFuture[0])).join();
+ } catch (Throwable t) {
+ LOG.error("Caught an exception in the main loop of compactor
cleaner, " +
+ StringUtils.stringifyException(t));
+ } finally {
+ if (handle != null) {
+ handle.releaseLocks();
+ }
}
+ // Now, go back to bed until it's time to do this again
+ long elapsedTime = System.currentTimeMillis() - startedAt;
+ if (elapsedTime < cleanerCheckInterval && !stop.get()) {
+ try {
+ Thread.sleep(cleanerCheckInterval - elapsedTime);
+ } catch (InterruptedException ie) {
Review comment:
Added the logging in catch.
##########
File path: ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
##########
@@ -274,6 +285,55 @@ public void droppedPartition() throws Exception {
ShowCompactResponse rsp = txnHandler.showCompact(new ShowCompactRequest());
Assert.assertEquals(0, rsp.getCompactsSize());
}
+
+ @Test
+ public void processCompactionCandidatesInParallel() throws Exception {
+ Table t = newTable("default", "camipc", true);
+ List<Partition> partitions = new ArrayList<>();
+ Partition p = null;
+ for(int i=0; i<10; i++) {
+ p = newPartition(t, "today" + i);
+
+ addBaseFile(t, p, 20L, 20);
+ addDeltaFile(t, p, 21L, 22L, 2);
+ addDeltaFile(t, p, 23L, 24L, 2);
+ addDeltaFile(t, p, 21L, 24L, 4);
+ partitions.add(p);
+ }
+ burnThroughTransactions("default", "camipc", 25);
Review comment:
Done.
----------------------------------------------------------------
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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]