You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/15 00:21:44 UTC

[GitHub] [incubator-pinot] sajjad-moradi opened a new pull request #5705: Add error message to broker response in case of broker send error

sajjad-moradi opened a new pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705


   On Pinot Broker, the incoming request gets written into socket channels of the target Servers. This happens on `QueryRouter.submitQuery(...)` function. If any exception occurs during  submitQuery for any reason like connection refused to one of the servers, sending requests to the remaining servers are abandoned and the partial responses from already successful sent requests are returned in BrokerResponse with no indication of the exception.
   Although partial response is acceptable, there should be an indication of such problem in broker response to make life easier for ppl debugging this issue. Recently there was an incident where, for a specific use case, broker response was empty (no partial response) and there was no exception returned, while data was available on Pinot Servers. After good amount of time going through logs, this issue was discovered with exception being connection refused by only one faulty server.
   This PR adds the exception to BrokerReponse for easier debugging.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] sajjad-moradi commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
sajjad-moradi commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r456057414



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java
##########
@@ -39,6 +39,8 @@
   private final CountDownLatch _countDownLatch;
   private final long _maxEndTimeMs;
 
+  private Exception _brokerRequestSendException;

Review comment:
       Good suggestion. Being on same thread might change in future due to some refactoring and it's good to have it as volatile.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] sajjad-moradi commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
sajjad-moradi commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r457762443



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       I don't have the write access to merge this PR. How can I get the write access?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r456136670



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       To send stack trace, we should use `getException(ProcessingException processingException, Exception exception)` instead of calling this method to construct the error message and use `getException(ProcessingException processingException, String errorMessage)`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
mayankshriv commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r455237265



##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
##########
@@ -47,6 +49,8 @@
 import org.apache.pinot.spi.env.PinotConfiguration;
 import org.apache.pinot.spi.utils.builder.TableNameBuilder;
 
+import static org.apache.pinot.common.exception.QueryException.BROKER_REQUEST_SEND_ERROR_CODE;

Review comment:
       Avoid static import.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r456241246



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       Sorry didn't realize they are different classes.
   (Optional) You might also want to keep the `errorType` information in the error message? If so, you can construct a `ProcessingException` first, then use it to construct the `QueryProcessingException`: `new QueryProcessingException(exception.getErrorCode(), exception.getMessage())`




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] sajjad-moradi commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
sajjad-moradi commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r456153839



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       `getException` method takes `ProcessingException` as its argument while we need to construct a `QueryProcesssingException` to pass it in broker response. That's why i did that refactoring.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r455410039



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java
##########
@@ -39,6 +39,8 @@
   private final CountDownLatch _countDownLatch;
   private final long _maxEndTimeMs;
 
+  private Exception _brokerRequestSendException;

Review comment:
       Make it volatile? (Although it is always set and read by the same thread)

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java
##########
@@ -105,4 +107,12 @@ void markServerDown(ServerRoutingInstance serverRoutingInstance) {
       markQueryFailed();
     }
   }
+
+  public Exception getBrokerRequestSendException() {
+    return _brokerRequestSendException;
+  }
+
+  public void setBrokerRequestSendException(Exception brokerRequestSendException) {

Review comment:
       (nit) Package private?

##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       Not sure if we should expose this method

##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
##########
@@ -114,6 +118,10 @@ protected BrokerResponse processBrokerRequest(long requestId, BrokerRequest orig
     brokerResponse.setNumServersQueried(numServersQueried);
     brokerResponse.setNumServersResponded(numServersResponded);
 
+    if (asyncQueryResponse.getBrokerRequestSendException() != null) {
+      String errorMsg = QueryException.getTruncatedStackTrace(asyncQueryResponse.getBrokerRequestSendException());
+      brokerResponse.addToExceptions(new QueryProcessingException(BROKER_REQUEST_SEND_ERROR_CODE, errorMsg));
+    }

Review comment:
       ```suggestion
       Exception brokerRequestSendException = asyncQueryResponse.getBrokerRequestSendException();
       if (brokerRequestSendException != null) {
         brokerResponse.addToExceptions(new QueryProcessingException(BROKER_REQUEST_SEND_ERROR_CODE, brokerRequestSendException));
       }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] sajjad-moradi commented on pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
sajjad-moradi commented on pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#issuecomment-658860655


   > An ECONNREFUSED situation (indicating that the remote server refused connection) means that the pinot-server process on the server is not running.
   > It will happen very quickly (the kernel will return the error), definitely much quicker than any response from other requests for other servers. So, when the broker gets notified of a connection request failure, it is likely the first response seen in the map of responses.
   > 
   > If the pinot-server process has gone down, then it is likely the case that we have not gotten notification from Helix as yet (otherwise, we would have taken down the server from the routing table and would not be routing to that server).
   > 
   > So, we are trying to address a window between the time the server went down, and the time we get notified by Helix.
   > 
   > True, Helix can sometimes take longer time to notify (depending on zookeeper speed).
   > 
   > It is good to note in the PR that this is the case we are addressing.
   
   That's true. That's the issue that we faced and we want to prevent that, but please note that we're catching not only connection refused exception but all types of exceptions in `submitQuery()` and adding the exception stack trace to the returned broker response will help debugging for other issues as well.
   ```java
       for (Map.Entry<ServerRoutingInstance, InstanceRequest> entry : requestMap.entrySet()) {
         ServerRoutingInstance serverRoutingInstance = entry.getKey();
         try {
           _serverChannels.sendRequest(serverRoutingInstance, entry.getValue());
           asyncQueryResponse.markRequestSubmitted(serverRoutingInstance);
         } catch (Exception e) {
           LOGGER.error("Caught exception while sending request {} to server: {}, marking query failed", requestId,
               serverRoutingInstance, e);
           _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_SEND_EXCEPTIONS, 1);
           asyncQueryResponse.setBrokerRequestSendException(e);
           asyncQueryResponse.markQueryFailed();
           break;
         }
       }
   ```


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang merged pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r456136984



##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
##########
@@ -114,6 +116,12 @@ protected BrokerResponse processBrokerRequest(long requestId, BrokerRequest orig
     brokerResponse.setNumServersQueried(numServersQueried);
     brokerResponse.setNumServersResponded(numServersResponded);
 
+    Exception brokerRequestSendException = asyncQueryResponse.getBrokerRequestSendException();
+    if (brokerRequestSendException != null) {
+      String errorMsg = QueryException.getTruncatedStackTrace(brokerRequestSendException);
+      brokerResponse
+          .addToExceptions(new QueryProcessingException(QueryException.BROKER_REQUEST_SEND_ERROR_CODE, errorMsg));

Review comment:
       ```suggestion
         brokerResponse
             .addToExceptions(new QueryProcessingException(QueryException.BROKER_REQUEST_SEND_ERROR_CODE, brokerRequestSendException));
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] sajjad-moradi commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
sajjad-moradi commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r457564370



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       `errorType`, which is basically `exception.getMessage()`, is always available in the first line of the stack trace. Since we keep the stack trace, we don't need to separately keep it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r456136984



##########
File path: pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java
##########
@@ -114,6 +116,12 @@ protected BrokerResponse processBrokerRequest(long requestId, BrokerRequest orig
     brokerResponse.setNumServersQueried(numServersQueried);
     brokerResponse.setNumServersResponded(numServersResponded);
 
+    Exception brokerRequestSendException = asyncQueryResponse.getBrokerRequestSendException();
+    if (brokerRequestSendException != null) {
+      String errorMsg = QueryException.getTruncatedStackTrace(brokerRequestSendException);
+      brokerResponse
+          .addToExceptions(new QueryProcessingException(QueryException.BROKER_REQUEST_SEND_ERROR_CODE, errorMsg));

Review comment:
       ```suggestion
         brokerResponse
             .addToExceptions(new QueryProcessingException(QueryException.BROKER_REQUEST_SEND_ERROR_CODE, brokerRequestSendException));
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] sajjad-moradi commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
sajjad-moradi commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r456059061



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       Why not? One example of its usefulness is in this PR as we need to truncate the stack trace and return it in broker response.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r457847609



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       @sajjad-moradi Merged the PR. I think you need to be a committer in order to merge PR. @mayankshriv @mcvsubbu Do we need to start a vote in order to make Sajjad a committer?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #5705: Add error message to broker response in case of broker send error

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #5705:
URL: https://github.com/apache/incubator-pinot/pull/5705#discussion_r457761728



##########
File path: pinot-common/src/main/java/org/apache/pinot/common/exception/QueryException.java
##########
@@ -125,8 +126,17 @@ public static void setMaxLinesOfStackTrace(int maxLinesOfStackTrace) {
   }
 
   public static ProcessingException getException(ProcessingException processingException, Exception exception) {
+    return getException(processingException, getTruncatedStackTrace(exception));
+  }
+
+  public static ProcessingException getException(ProcessingException processingException, String errorMessage) {
     String errorType = processingException.getMessage();
     ProcessingException copiedProcessingException = processingException.deepCopy();
+    copiedProcessingException.setMessage(errorType + ":\n" + errorMessage);
+    return copiedProcessingException;
+  }
+
+  public static String getTruncatedStackTrace(Exception exception) {

Review comment:
       The `errorType` I referred to here is `BrokerResponseSendError` which you put in the `QueryException.class` so that users don't need to look up the int error code (425) to understand the type of the error. The error message of the Exception will be:
   ```
     BrokerResponseSendError:
     {stacktrace}
   ```
   
   I already approved the change. These comments are just suggestions. You can merge it




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org