You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "imply-cheddar (via GitHub)" <gi...@apache.org> on 2023/02/14 02:26:33 UTC

[GitHub] [druid] imply-cheddar commented on a diff in pull request #13798: Introduce a DruidException for better user error messages

imply-cheddar commented on code in PR #13798:
URL: https://github.com/apache/druid/pull/13798#discussion_r1105214697


##########
sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java:
##########
@@ -90,7 +90,9 @@ AggregatorFactory createAggregatorFactory(String name, String fieldName, String
           case COMPLEX:
             return new StringFirstAggregatorFactory(name, fieldName, timeColumn, maxStringBytes);
           default:
-            throw new UnsupportedSQLQueryException("EARLIEST aggregator is not supported for '%s' type", type);
+            throw DruidException.user("EARLIEST aggregator is not supported for '%s' type", type)

Review Comment:
   These messages would be better adjusted to wrap in `[]` instead of `''` to align with convention.  They are solving the same need though.



##########
sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java:
##########
@@ -219,12 +226,14 @@ public Aggregation toDruidAggregation(
         );
         break;
       default:
-        throw new IAE(
-            "aggregation[%s], Invalid number of arguments[%,d] to [%s] operator",
-            aggregatorName,
-            args.size(),
-            aggregatorType.name()
-        );
+        throw DruidException.user(
+              "aggregation [%s], Invalid number of arguments %,d to [%s] operator",

Review Comment:
   We shouldn't re-format this.  Or, if we are going to reformat, we should move the final `[%s]` to immediately after the word "operator"



##########
sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlParserUtils.java:
##########
@@ -229,18 +230,18 @@ public static List<String> validateQueryAndConvertToIntervals(
     List<Interval> intervals = filtration.getIntervals();
 
     if (filtration.getDimFilter() != null) {
-      throw new ValidationException("Only " + ColumnHolder.TIME_COLUMN_NAME + " column is supported in OVERWRITE WHERE clause");
+      throw DruidException.userError("Only " + ColumnHolder.TIME_COLUMN_NAME + " column is supported in OVERWRITE WHERE clause");

Review Comment:
   If we are cleaning up the messages, making these use format specifiers instead of concatenation would be good.



##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java:
##########
@@ -164,7 +167,10 @@ private SqlStatementHandler createHandler(final SqlNode node) throws ValidationE
     if (query.isA(SqlKind.QUERY)) {
       return new QueryHandler.SelectHandler(handlerContext, query, explain);
     }
-    throw new ValidationException(StringUtils.format("Cannot execute [%s].", node.getKind()));
+    throw DruidException.user("Unsupported SQL statement")
+        .context("Statement kind", node.getKind())
+        .context(DruidException.ERROR_CODE, QueryException.PLAN_VALIDATION_FAILED_ERROR_CODE)
+        .build();

Review Comment:
   Moving the kind to the context here very possibly hides valuable information that was intended to be in the message.  We shouldn't think of context items as replacing anything in the message.  They can augment with extra things that we explicitly don't want in the message, but "it exists in the context" shouldn't be a reason that it cannot exist in the message.



##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,359 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.error;
+
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.
+ * Not needed for purely internal exceptions thrown and caught within Druid itself.
+ * There are categories of error that determine the general form of corrective
+ * action, and also determine HTTP (or other API) status codes.
+ * <p>
+ * Druid exceptions can contain context. Use the context for details, such as
+ * file names, query context variables, symbols, etc. This allows the error
+ * message itself to be simple. Context allows consumers to filter out various
+ * bits of information that a site does not wish to expose to the user, while
+ * still logging the full details. Typical usage:
+ * <pre><code>
+ * if (something_is_wrong) {
+ *   throw DruidException.user("File not found")
+ *       .context("File name", theFile.getName())
+ *       .context("Directory", theFile.getParent())
+ *       .build();
+ * }
+ * </code></pre>
+ * <p>
+ * Exceptions are immutable. In many cases, an error is thrown low in the code,
+ * bit context is known at a higher level. In this case, the higher code should
+ * catch the exception, convert back to a builder, add context, and throw the
+ * new exception. The original call stack is maintained. Example:
+ * <pre><code>
+ * catch (DruidExceptin e) {
+ *   throw e.toBuilder().
+ *       .context("File name", theFile.getName())
+ *       .context("Directory", theFile.getParent())
+ *       .build();
+ * }
+ * </code></pre>
+ */
+public class DruidException extends RuntimeException
+{
+  public enum ErrorType
+  {
+    /**
+     * General case of an error due to something the user asked to do in an REST
+     * request. Translates to an HTTP status 400 (BAD_REQUET) for a REST call
+     * (or the equivalent for other APIs.)
+     */
+    USER,
+
+    /**
+     * Special case of a user error where a resource is not found and we wish
+     * to return a 404 (NOT_FOUND) HTTP status (or the equivalent for other
+     * APIs.)
+     */
+    NOT_FOUND,
+
+    /**
+     * Error due to a problem beyond the user's control, such as an assertion
+     * failed, unsupported operation, etc. These indicate problems with the software
+     * where the fix is either a workaround or a bug fix. Such error should only
+     * be raised for "should never occur" type situations.
+     */
+    SYSTEM,
+
+    /**
+     * Error for a resource limit: memory, CPU, slots or so on. The workaround is
+     * generally to try later, get more resources, reduce load or otherwise resolve
+     * the resource pressure issue.
+     */
+    RESOURCE,
+
+    /**
+     * Similar to RESOURCE, except indicates a timeout, perhaps due to load, due
+     * to an external system being unavailable, etc.
+     */
+    TIMEOUT,
+
+    /**
+     * Error in configuration. Indicates that the administrator made a mistake during
+     * configuration or setup. The solution is for the administrator (not the end user)
+     * to resolve the issue.
+     */
+    CONFIG,
+
+    /**
+     * Indicates a network error of some kind: intra-Druid, client-to-Druid,
+     * Druid-to-external system, etc. Generally the end user cannot fix these errors:
+     * it requires a DevOps person to resolve.
+     */
+    NETWORK
+  };

Review Comment:
   These have a mixing of persona-defined types and problem-defined types.  I think it will be potentially hard to be correct in picking the right type all the time.  What do you think of having a persona-defined type ("this is end-user facing, admin facing, Druid developer facing") and then another enumeration to indicate what kind of response code is generally expected to occur.
   
   This would get developers to think of 2 things while writing their exception messages:
   
   1) The persona that they are writing the message for
   2) What kind of error code they expect to be given back.



##########
sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java:
##########
@@ -2692,9 +2692,12 @@ public void testGroupByInvalidPath()
         + "SUM(cnt) "
         + "FROM druid.nested GROUP BY 1",
         (expected) -> {
-          expected.expect(UnsupportedSQLQueryException.class);
+          expected.expect(DruidException.class);
           expected.expectMessage(
-              "Cannot use [JSON_VALUE_VARCHAR]: [Bad format, '.array.[1]' is not a valid JSONPath path: must start with '$']");
+              "Cannot use JSON_VALUE_VARCHAR\n" +
+              "Error Code: Unsupported operation\n" +
+              "Cause: Bad format, '.array.[1]' is not a valid JSONPath path: must start with '$'"

Review Comment:
   This change in test highlights something taht we definitely don't want to happen: the message of the exception is losing valuable information about what actual bad thing happened.  On top of that, if we happen to be serializing it out with all of the context, then we get the information back, but now it's new-line delimited and new-line delimited exception messages are very annoying to deal with when trying to operate a system.
   
   Instead of generating a new-line delimited error message, when we are actively trying to log the full context, it should all be on the same line as the original error message.  And, the error message itself should be considered a self-contained, meaningful thing that can be used by its intended audience without anything from the context needing to also exist.



##########
server/src/main/java/org/apache/druid/server/security/PreResponseAuthorizationCheckFilter.java:
##########
@@ -90,7 +89,7 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo
       // (e.g. OverlordServletProxy), so this is not implemented for now.
       handleAuthorizationCheckError(
           StringUtils.format(
-              "Request did not have an authorization check performed, original response status[%s].",
+              "Request did not have an authorization check performed, original response status: %s",

Review Comment:
   Please do not remove interpolations from the `[]`.  It's an established standard convention for generating log messages and exception messages.



##########
sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java:
##########
@@ -219,12 +226,14 @@ public Aggregation toDruidAggregation(
         );
         break;
       default:
-        throw new IAE(
-            "aggregation[%s], Invalid number of arguments[%,d] to [%s] operator",
-            aggregatorName,
-            args.size(),
-            aggregatorType.name()
-        );
+        throw DruidException.user(
+              "aggregation [%s], Invalid number of arguments %,d to [%s] operator",
+              aggregatorName,
+              args.size(),
+              aggregatorType.name()
+             )
+            .context(DruidException.ERROR_CODE, QueryException.PLAN_VALIDATION_FAILED_ERROR_CODE)

Review Comment:
   I think it's weird to put this on as a context parameter.  I interpret this to mean that we are explicitly trying to override the response error code, right?  If we have an error code like this, make it first-class on the exception.



##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java:
##########
@@ -91,7 +94,10 @@ public void reduce(
           // as a primitive long/float/double.
           // ExprEval.isNumericNull checks whether the parsed primitive value is null or not.
           if (!constExp.getType().isNullable() && exprResult.isNumericNull()) {
-            throw new UnsupportedSQLQueryException("Illegal DATE constant: %s", constExp);
+            throw DruidException.user("Illegal DATE constant")

Review Comment:
   Here again meaningful values (the date) that should get back to the user are potentially going to be ellided because they are in the context.



-- 
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: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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