[
https://issues.apache.org/jira/browse/HIVE-23324?focusedWorklogId=463914&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-463914
]
ASF GitHub Bot logged work on HIVE-23324:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 29/Jul/20 07:43
Start Date: 29/Jul/20 07:43
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_r461306773
##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -66,53 +69,62 @@
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);
- }
-
- @Override
- public void run() {
if (cleanerCheckInterval == 0) {
Review comment:
Yes, this is redundant. Removed it.
##########
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:
I don't think it is redundant.
In initiator, the try-catch (catching throwable) is applied on the complete
`while` loop. Since `thread.sleep` is inside while loop, it was redundant in
initiator. (Though, `sleep` throws an `Interrupted` exception, we will get out
of while loop and initiator will exit).
In case of cleaner, the try-catch (catching throwable) is applied only on
the main logic for cleaning directories. This is inside `while` loop (as
compared to complete `while` loop for initiator). Here, even if `sleep` throws
`Interrupted` exception, Cleaner won't exit because of a separate try-catch
statement inside `while` loop.
##########
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:
There is no catch in Cleaner, its just try-finally.
----------------------------------------------------------------
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]
Issue Time Tracking
-------------------
Worklog Id: (was: 463914)
Time Spent: 10h 20m (was: 10h 10m)
> 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: 10h 20m
> 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)