This is an automated email from the ASF dual-hosted git repository.

snlee pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 0bdb89c  Add broker request info to the error msg in combine operator. 
(#5443)
0bdb89c is described below

commit 0bdb89c4d4e8ecd4352a60a66b23806d7e39219a
Author: Seunghyun Lee <[email protected]>
AuthorDate: Tue May 26 15:29:40 2020 -0700

    Add broker request info to the error msg in combine operator. (#5443)
    
    * Add broker request info to the error msg in combine operator.
    
    Current log indicates exception/timeout in combine operator but
    this does not give much information for debugging. Adding broker
    request to the log will help for debugging.
    
    * Add broker requests to the error log in other combine operators
---
 .../org/apache/pinot/core/operator/CombineGroupByOperator.java    | 8 +++++---
 .../apache/pinot/core/operator/CombineGroupByOrderByOperator.java | 8 +++++---
 .../main/java/org/apache/pinot/core/operator/CombineOperator.java | 6 +++---
 3 files changed, 13 insertions(+), 9 deletions(-)

diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineGroupByOperator.java
 
b/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineGroupByOperator.java
index a7bb5c2..4ce2a6e 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineGroupByOperator.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineGroupByOperator.java
@@ -174,8 +174,8 @@ public class CombineGroupByOperator extends 
BaseOperator<IntermediateResultsBloc
           } catch (EarlyTerminationException e) {
             // Early-terminated because query times out or is already satisfied
           } catch (Exception e) {
-            LOGGER.error("Exception processing CombineGroupBy for index {}, 
operator {}", index,
-                _operators.get(index).getClass().getName(), e);
+            LOGGER.error("Exception processing CombineGroupBy for index {}, 
operator {}, brokerRequest {}", index,
+                _operators.get(index).getClass().getName(), _brokerRequest, e);
             
mergedProcessingExceptions.add(QueryException.getException(QueryException.QUERY_EXECUTION_ERROR,
 e));
           } finally {
             operatorLatch.countDown();
@@ -189,7 +189,9 @@ public class CombineGroupByOperator extends 
BaseOperator<IntermediateResultsBloc
       boolean opCompleted = operatorLatch.await(_timeOutMs, 
TimeUnit.MILLISECONDS);
       if (!opCompleted) {
         // If this happens, the broker side should already timed out, just log 
the error and return
-        String errorMessage = "Timed out while combining group-by results 
after " + _timeOutMs + "ms";
+        String errorMessage =
+            String.format("Timed out while combining group-by results after 
%dms, brokerRequest = %s", _timeOutMs,
+                _brokerRequest);
         LOGGER.error(errorMessage);
         return new IntermediateResultsBlock(new 
TimeoutException(errorMessage));
       }
diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineGroupByOrderByOperator.java
 
b/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineGroupByOrderByOperator.java
index c45f8ba..610218a 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineGroupByOrderByOperator.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineGroupByOrderByOperator.java
@@ -189,8 +189,8 @@ public class CombineGroupByOrderByOperator extends 
BaseOperator<IntermediateResu
           } catch (EarlyTerminationException e) {
             // Early-terminated because query times out or is already satisfied
           } catch (Exception e) {
-            LOGGER.error("Exception processing CombineGroupByOrderBy for index 
{}, operator {}", index,
-                _operators.get(index).getClass().getName(), e);
+            LOGGER.error("Exception processing CombineGroupByOrderBy for index 
{}, operator {}, brokerRequest {}", index,
+                _operators.get(index).getClass().getName(), _brokerRequest, e);
             
mergedProcessingExceptions.add(QueryException.getException(QueryException.QUERY_EXECUTION_ERROR,
 e));
           } finally {
             operatorLatch.countDown();
@@ -204,7 +204,9 @@ public class CombineGroupByOrderByOperator extends 
BaseOperator<IntermediateResu
       boolean opCompleted = operatorLatch.await(_timeOutMs, 
TimeUnit.MILLISECONDS);
       if (!opCompleted) {
         // If this happens, the broker side should already timed out, just log 
the error and return
-        String errorMessage = "Timed out while combining group-by results 
after " + _timeOutMs + "ms";
+        String errorMessage =
+            String.format("Timed out while combining group-by results after 
%dms, brokerRequest = %s", _timeOutMs,
+                _brokerRequest);
         LOGGER.error(errorMessage);
         return new IntermediateResultsBlock(new 
TimeoutException(errorMessage));
       }
diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineOperator.java 
b/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineOperator.java
index 7b559cc..09a9712 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineOperator.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/operator/CombineOperator.java
@@ -168,13 +168,13 @@ public class CombineOperator extends 
BaseOperator<IntermediateResultsBlock> {
     try {
       mergedBlock = mergedBlockFuture.get(endTimeMs - 
System.currentTimeMillis(), TimeUnit.MILLISECONDS);
     } catch (InterruptedException e) {
-      LOGGER.error("Caught InterruptedException.", e);
+      LOGGER.error("Caught InterruptedException. (brokerRequest = {})", 
_brokerRequest, e);
       mergedBlock = new 
IntermediateResultsBlock(QueryException.getException(QueryException.FUTURE_CALL_ERROR,
 e));
     } catch (ExecutionException e) {
-      LOGGER.error("Caught ExecutionException.", e);
+      LOGGER.error("Caught ExecutionException. (brokerRequest = {})", 
_brokerRequest, e);
       mergedBlock = new 
IntermediateResultsBlock(QueryException.getException(QueryException.MERGE_RESPONSE_ERROR,
 e));
     } catch (TimeoutException e) {
-      LOGGER.error("Caught TimeoutException", e);
+      LOGGER.error("Caught TimeoutException. (brokerRequest = {})", 
_brokerRequest, e);
       mergedBlockFuture.cancel(true);
       mergedBlock =
           new 
IntermediateResultsBlock(QueryException.getException(QueryException.EXECUTION_TIMEOUT_ERROR,
 e));


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to