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

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


The following commit(s) were added to refs/heads/master by this push:
     new e3f2a3141d Reduce logging spam (#15533)
e3f2a3141d is described below

commit e3f2a3141df5b089545585600c666722c525c379
Author: Alberto Bastos <alberto.var...@startree.ai>
AuthorDate: Tue Apr 15 14:36:06 2025 +0200

    Reduce logging spam (#15533)
---
 .../broker/requesthandler/MultiStageBrokerRequestHandler.java |  2 +-
 .../org/apache/pinot/query/mailbox/GrpcSendingMailbox.java    |  5 ++---
 .../apache/pinot/query/mailbox/InMemorySendingMailbox.java    | 11 +++++++----
 .../pinot/query/mailbox/channel/MailboxContentObserver.java   |  8 +++++---
 .../org/apache/pinot/query/runtime/blocks/ErrorMseBlock.java  | 11 ++++++++++-
 .../pinot/query/runtime/operator/MailboxSendOperator.java     |  2 +-
 6 files changed, 26 insertions(+), 13 deletions(-)

diff --git 
a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java
 
b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java
index 44e74ccc70..dc2a2bd604 100644
--- 
a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java
+++ 
b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java
@@ -228,6 +228,7 @@ public class MultiStageBrokerRequestHandler extends 
BaseBrokerRequestHandler {
   ///     - A table not authorized to read is used
   ///     - An exception during function execution due to errors in the data
   ///       (ie a division by zero or casting an illegal string as int)
+  ///     - Query is too heavy and reaches the allowed timeout.
   ///   - The error message will be sent to the user and the error messages 
will be logged without stack trace.
   /// 3. With yellow error: The request failed in a way that is controlled but 
probably internal.
   ///   - The error message will be sent to the user and the error message 
will be logged with stack trace.
@@ -692,7 +693,6 @@ public class MultiStageBrokerRequestHandler extends 
BaseBrokerRequestHandler {
   public static boolean isYellowError(QueryException e) {
     switch (e.getErrorCode()) {
       case QUERY_SCHEDULING_TIMEOUT:
-      case EXECUTION_TIMEOUT:
       case INTERNAL:
       case UNKNOWN:
       case MERGE_RESPONSE:
diff --git 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java
 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java
index 52ff58fc7e..13f895c669 100644
--- 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java
+++ 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/GrpcSendingMailbox.java
@@ -131,9 +131,8 @@ public class GrpcSendingMailbox implements SendingMailbox {
     try {
       String msg = t != null ? t.getMessage() : "Unknown";
       // NOTE: DO NOT use onError() because it will terminate the stream, and 
receiver might not get the callback
-      _contentObserver.onNext(toMailboxContent(
-          ErrorMseBlock.fromException(new RuntimeException("Cancelled by 
sender with exception: " + msg, t)),
-          List.of()));
+      _contentObserver.onNext(toMailboxContent(ErrorMseBlock.fromError(
+          QueryErrorCode.QUERY_CANCELLATION, "Cancelled by sender with 
exception: " + msg), List.of()));
       _contentObserver.onCompleted();
     } catch (Exception e) {
       // Exception can be thrown if the stream is already closed, so we simply 
ignore it
diff --git 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java
 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java
index d3e1afa879..43cfb7bf9f 100644
--- 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java
+++ 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/InMemorySendingMailbox.java
@@ -28,6 +28,8 @@ import org.apache.pinot.query.runtime.blocks.MseBlock;
 import org.apache.pinot.query.runtime.operator.MailboxSendOperator;
 import org.apache.pinot.segment.spi.memory.DataBuffer;
 import org.apache.pinot.spi.exception.QueryCancelledException;
+import org.apache.pinot.spi.exception.QueryErrorCode;
+import org.apache.pinot.spi.exception.QueryException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -87,9 +89,10 @@ public class InMemorySendingMailbox implements 
SendingMailbox {
       case CANCELLED:
         throw new QueryCancelledException(String.format("Mailbox: %s already 
cancelled from upstream", _id));
       case ERROR:
-        throw new RuntimeException(String.format("Mailbox: %s already errored 
out (received error block before)", _id));
+        throw new QueryException(QueryErrorCode.INTERNAL, String.format(
+            "Mailbox: %s already errored out (received error block before)", 
_id));
       case TIMEOUT:
-        throw new TimeoutException(
+        throw new QueryException(QueryErrorCode.EXECUTION_TIMEOUT,
             String.format("Timed out adding block into mailbox: %s with 
timeout: %dms", _id, timeoutMs));
       case EARLY_TERMINATED:
         _isEarlyTerminated = true;
@@ -114,8 +117,8 @@ public class InMemorySendingMailbox implements 
SendingMailbox {
       _receivingMailbox = _mailboxService.getReceivingMailbox(_id);
     }
     _receivingMailbox.setErrorBlock(
-        ErrorMseBlock.fromException(new RuntimeException("Cancelled by sender 
with exception: " + t.getMessage(), t)),
-        Collections.emptyList());
+        ErrorMseBlock.fromException(new QueryCancelledException(
+            "Cancelled by sender with exception: " + t.getMessage())), 
Collections.emptyList());
   }
 
   @Override
diff --git 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentObserver.java
 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentObserver.java
index cf7de154a0..04b155e19f 100644
--- 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentObserver.java
+++ 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/mailbox/channel/MailboxContentObserver.java
@@ -22,12 +22,14 @@ import io.grpc.Context;
 import io.grpc.stub.StreamObserver;
 import java.nio.ByteBuffer;
 import java.util.Collections;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import org.apache.pinot.common.proto.Mailbox.MailboxContent;
 import org.apache.pinot.common.proto.Mailbox.MailboxStatus;
 import org.apache.pinot.query.mailbox.MailboxService;
 import org.apache.pinot.query.mailbox.ReceivingMailbox;
 import org.apache.pinot.query.runtime.blocks.ErrorMseBlock;
+import org.apache.pinot.spi.exception.QueryErrorCode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -113,9 +115,9 @@ public class MailboxContentObserver implements 
StreamObserver<MailboxContent> {
   public void onError(Throwable t) {
     LOGGER.warn("Error on receiver side", t);
     if (_mailbox != null) {
-      _mailbox.setErrorBlock(
-          ErrorMseBlock.fromException(new RuntimeException("Cancelled by 
sender", t)),
-          Collections.emptyList());
+      String msg = t != null ? t.getMessage() : "Unknown";
+      _mailbox.setErrorBlock(ErrorMseBlock.fromError(
+          QueryErrorCode.QUERY_CANCELLATION, "Cancelled by sender with 
exception: " + msg), List.of());
     } else {
       LOGGER.error("Got error before mailbox is set up", t);
     }
diff --git 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/ErrorMseBlock.java
 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/ErrorMseBlock.java
index 433015026d..7d4b661ff6 100644
--- 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/ErrorMseBlock.java
+++ 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/blocks/ErrorMseBlock.java
@@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
 import java.util.Collections;
 import java.util.EnumMap;
 import java.util.Map;
+import javax.validation.constraints.NotNull;
 import org.apache.pinot.common.datablock.DataBlockUtils;
 import org.apache.pinot.common.response.ProcessingException;
 import org.apache.pinot.spi.exception.QueryErrorCode;
@@ -42,7 +43,6 @@ public class ErrorMseBlock implements MseBlock.Eos {
   }
 
   public static ErrorMseBlock fromException(Exception e) {
-    String errorMessage = DataBlockUtils.extractErrorMsg(e);
     QueryErrorCode errorCode;
     if (e instanceof QueryException) {
       errorCode = ((QueryException) e).getErrorCode();
@@ -51,9 +51,18 @@ public class ErrorMseBlock implements MseBlock.Eos {
     } else {
       errorCode = QueryErrorCode.UNKNOWN;
     }
+    String errorMessage = shouldIncludeStackTrace(errorCode) ? 
DataBlockUtils.extractErrorMsg(e) : e.getMessage();
     return new ErrorMseBlock(Collections.singletonMap(errorCode, 
errorMessage));
   }
 
+  public static ErrorMseBlock fromError(QueryErrorCode errorCode, String 
errorMessage) {
+    return new ErrorMseBlock(Collections.singletonMap(errorCode, 
errorMessage));
+  }
+
+  private static boolean shouldIncludeStackTrace(@NotNull QueryErrorCode 
errorCode) {
+    return QueryErrorCode.UNKNOWN.equals(errorCode);
+  }
+
   @Override
   public boolean isError() {
     return true;
diff --git 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java
 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java
index 8e4a3f2953..88a0b16b94 100644
--- 
a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java
+++ 
b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java
@@ -225,7 +225,7 @@ public class MailboxSendOperator extends MultiStageOperator 
{
     } catch (Exception e) {
       ErrorMseBlock errorBlock = ErrorMseBlock.fromException(e);
       try {
-        LOGGER.error("Exception while transferring data on opChain: {}", 
_context.getId(), e);
+        LOGGER.error("Exception while transferring data on opChain: {}", 
_context.getId());
         sendEos(errorBlock);
       } catch (Exception e2) {
         LOGGER.error("Exception while sending error block.", e2);


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

Reply via email to