You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "paul-rogers (via GitHub)" <gi...@apache.org> on 2023/02/18 01:56:35 UTC

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

paul-rogers commented on code in PR #13798:
URL: https://github.com/apache/druid/pull/13798#discussion_r1110438887


##########
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:
   As it turns out, the topics above are _supposed_ to represent personas. The `USER` category is something that (we believe) the user caused by virtue of something just requested. Most likely the user has to take action.
   
   A `CONFIG` error is targeted at the sys admin: they configured something wrong and they have to change a config (or align an external system with the config) to fix the error.
   
   A `SYSTEM` error is targeted at us: something is wrong in the code. The user will get the error, but all they can do is turn around, forward the message to us and ask us to fix whatever it is that went wrong.
   
   A `NETWORK` error _could_ be the user (they gave us a bogus HTTP input source URL), could be an admin (they didn't open a port required by Druid) or a generic error (someone with a backhoe just cut the fiber optic link.) If we can figure out which if these cases is occurring, we could better target the error. Do you know of a way to know this? Are our network errors clearly delimited split out in the code based on who should take action?
   
   For others it gets even muddier. A `RESOURCE` error can be seen as a user error ("hey! Don't query so many rows"), a load error ("there just isn't enough load to go around: go away and come back later."), a config issue ("1 GB is a bit low for this workload"), etc. Any suggestions for how we'd know, in the code, to whom to attribute the cause of the lack of resources?
   
   Given all this, the categories were chosen as a compromise between the _probable_ person who can fix things, and what we _actually_ know in the code.
   
   Part of the reason for this PR is to see if we can improve the categories. Persona is one: `USER`, `SECURITY_ADMIN`, `CLUSTER_ADMIN`, `YOUR_NETWORK_PROVIDER`, `THE_BOSS_WHO_APPROVES_RESOURCES`, etc. I'm being silly, but what are the right personas? How would a person in the code know which persona is the target?
   
   Any third solution that captures both ideas (persona and fits what we know in the code)?



##########
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:
   This is a hard one! Our error codes, at present, are somewhat random. We have multiple codes for some items, no codes for others. (E.g. `QUERY_UNSUPPORTED_ERROR_CODE` `SQL_QUERY_UNSUPPORTED_ERROR_CODE`. But nothing at all for "not found".). The current system didn't seem like something that can build upon.
   
   That said, it probably does make sense to add additional methods to `DruidException` for the obvious cases: `validationError()`, `unsupported()`, etc. for the planner errors.
   
   Also, after playing around with the code, it seems to make more sense for the error code to be a top-level exception attribute, not a context property.



##########
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:
   See comment in the main discussion. This is a convention in _Druid_. It is not a convention elsewhere. We're mainly focused on end users here. Will your typical end user expect values in brackets? Or, should we use this format just for internal messages and log messages? Maybe omit for `USER` messages, include for all others. Something else?



##########
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:
   Part of the goal of this project is to translate messages in "developerese" into English. Rewritten to:
   
   `"%s expects 1 or 2 arguments but found %d"`
   
   The notion is that the SQL user knows the operator which they wrote in the SQL. That operator takes 1 or 2 arguments. The user provided more, which is not valid. The internal aggregation name is not known to the SQL user, so the revised code omits that name.
   
   Example:
   
   > LATEST expects 1 or 2 arguments but found 3
   
   To touch on the raging brackets debate, would the above be clearer to an SQL user if it was formatted as:
   
   > [LATEST] expects 1 or 2 arguments but found [3]
   



##########
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:
   See discussion in main conversation. Once we decide on a style, I'll update all the revised error messages.



##########
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:
   Reverted the change. We can focus our efforts where we have poor errors. This one seems fine as is.



##########
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:
   This pass included some experiments to see what worked and what didn't. I agree that, in this case, using the context didn't work. Revised to:
   
   > Unsupported SQL statement %s



-- 
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