[ https://issues.apache.org/jira/browse/HIVE-25746?focusedWorklogId=715641&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-715641 ]
ASF GitHub Bot logged work on HIVE-25746: ----------------------------------------- Author: ASF GitHub Bot Created on: 26/Jan/22 12:25 Start Date: 26/Jan/22 12:25 Worklog Time Spent: 10m Work Description: vcsomor commented on a change in pull request #2974: URL: https://github.com/apache/hive/pull/2974#discussion_r792584766 ########## File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/MetaStoreCompactorThread.java ########## @@ -128,9 +132,40 @@ protected static long updateCycleDurationMetric(String metric, long startedAt) { long elapsed = System.currentTimeMillis() - startedAt; LOG.debug("Updating {} metric to {}", metric, elapsed); Metrics.getOrCreateGauge(metric) - .set((int)elapsed); + .set((int) elapsed); return elapsed; } return 0; } + + @VisibleForTesting + protected static void waitAllAsyncTask(List<CompletableFuture<Void>> tasks) throws AsyncTaskCompletionException { + List<Throwable> exceptions = new ArrayList<>(); + for (CompletableFuture<Void> task : tasks) { + try { + task.join(); Review comment: Which thread safe list implementation do you prefer in this case? -- 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: gitbox-unsubscr...@hive.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 715641) Time Spent: 40m (was: 0.5h) > Compaction Failure Counter counted incorrectly > ---------------------------------------------- > > Key: HIVE-25746 > URL: https://issues.apache.org/jira/browse/HIVE-25746 > Project: Hive > Issue Type: Bug > Components: Metastore > Affects Versions: 4.0.0 > Reporter: Viktor Csomor > Assignee: Viktor Csomor > Priority: Minor > Labels: pull-request-available > Time Spent: 40m > Remaining Estimate: 0h > > The count of the below metrics counted incorrectly upon an exception. > - {{compaction_initator_failure_counter}} > - {{compaction_cleaner_failure_counter}} > Reasoning: > In the {{Initator}}/{{Cleaner}} class creates a list of {{CompletableFuture}} > which {{Runnable}} core exception is being wrapped to {{RuntimeExceptions}}. > The below code-snippet waits all cleaners to complete (Initiators does it > similarly). > {code:java} > try { > .... > for (CompactionInfo compactionInfo : readyToClean) { > > cleanerList.add(CompletableFuture.runAsync(CompactorUtil.ThrowingRunnable.unchecked(() > -> > clean(compactionInfo, cleanerWaterMark, > metricsEnabled)), cleanerExecutor)); > } > CompletableFuture.allOf(cleanerList.toArray(new > CompletableFuture[0])).join(); > } > } catch (Throwable t) { > // the lock timeout on AUX lock, should be ignored. > if (metricsEnabled && handle != null) { > failuresCounter.inc(); > } > {code} > If the {{CompleteableFututre#join}} throws an Exception then the failure > counter is incremented. > Docs: > {code} > /** > * Returns the result value when complete, or throws an > * (unchecked) exception if completed exceptionally. To better > * conform with the use of common functional forms, if a > * computation involved in the completion of this > * CompletableFuture threw an exception, this method throws an > * (unchecked) {@link CompletionException} with the underlying > * exception as its cause. > * > * @return the result value > * @throws CancellationException if the computation was cancelled > * @throws CompletionException if this future completed > * exceptionally or a completion computation threw an exception > */ > public T join() { > Object r; > return reportJoin((r = result) == null ? waitingGet(false) : r); > } > {code} > (!) Let's suppose we have 10 cleaners and the 2nd throws an exception. The > {{catch}} block will be initiated and the {{failuresCounter}} will be > incremented. If there is any consecutive error amongst the remaining cleaners > the counter won't be incremented. -- This message was sent by Atlassian Jira (v8.20.1#820001)