[ 
https://issues.apache.org/jira/browse/HIVE-25746?focusedWorklogId=715709&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-715709
 ]

ASF GitHub Bot logged work on HIVE-25746:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 26/Jan/22 14:18
            Start Date: 26/Jan/22 14:18
    Worklog Time Spent: 10m 
      Work Description: deniskuzZ commented on a change in pull request #2974:
URL: https://github.com/apache/hive/pull/2974#discussion_r792681772



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -155,18 +159,28 @@ public void run() {
             // when min_history_level is finally dropped, than every HMS will 
commit compaction the new way
             // and minTxnIdSeenOpen can be removed and minOpenTxnId can be 
used instead.
             for (CompactionInfo compactionInfo : readyToClean) {
-              
cleanerList.add(CompletableFuture.runAsync(ThrowingRunnable.unchecked(
-                  () -> clean(compactionInfo, cleanerWaterMark, 
metricsEnabled)), cleanerExecutor));
+              String tableName = compactionInfo.getFullTableName();
+              String partition = compactionInfo.getFullPartitionName();
+              CompletableFuture<Void> asyncJob =
+                  CompletableFuture.runAsync(
+                          ThrowingRunnable.unchecked(() -> 
clean(compactionInfo, cleanerWaterMark, metricsEnabled)),
+                          cleanerExecutor)
+                      .exceptionally(t -> {
+                        cleanerErrors.incrementAndGet();
+                        LOG.error("Error during the cleaning the table {} / 
partition {}", tableName, partition, t);
+                        return null;
+                      });
+              cleanerList.add(asyncJob);
             }
             CompletableFuture.allOf(cleanerList.toArray(new 
CompletableFuture[0])).join();
+
+            if (metricsEnabled && handle != null) {
+              failuresCounter.inc(cleanerErrors.get());

Review comment:
       i don't think this is correct as cleanerErrors is a global variable and 
would be incremented on every 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.

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: 715709)
    Time Spent: 1h 10m  (was: 1h)

> 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: 1h 10m
>  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)

Reply via email to