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/03/31 06:07:57 UTC

[GitHub] [druid] imply-cheddar opened a new pull request, #14004: Errors take 3

imply-cheddar opened a new pull request, #14004:
URL: https://github.com/apache/druid/pull/14004

   This is the 3rd take at creating a unified Exception in the DruidException.  Given that I had input on the previous two attempts, I figured I'd take the next stab at offering something that matches the input that I had provided.
   
   DruidException has class level javadoc discussing how it's all setup and why, so I would recommend starting with that class-level javadoc and then looking at some of the points where DruidExceptions are created (DruidPlanner and various of the other places).
   
   There is a compatibility layer in this PR that tries to continue mimicing the old way that QueryException was getting serialized out, but the current PR treats that as best-effort.  Specifically, anything that started its life as a DruidException will be an "unknown" error type when serialized back as a QueryException.  The error message should make it through, but the type characterization will be different.  This should only happen in the middle of roll-forward/roll-back processes, so it will be short-lived and, given that it preserves the message, is hopefully a decent compromise.
   
   Additionally, as I was adjusting tests, I got tired of the SQL tests taking roughly 5 minutes to run through the entire suite, so I looked into why it was doing that and through moving some initialization around and fixing some time-based async tests to be based on latches instead, was able to get that timing down to ~1.5 minutes, so some of those changes are mixed in here as well.
   
   #### Release note
   
   A new unified exception for surfacing errors to users has been introduced.  It is partially compatible with the old way of reporting error messages, response codes remain the same, all fields that previously existed on the response will continue to exist and be populated, including the errorMessage.  But some error messages have changed to be more consumable by humans and some cases will have the message restructured.  Any clients that are actively looking at the error message to make decisions on what to do should explore the changes to the error messages that they care about, but if all a client looks at is the response code, there should be no impact.  
   
   This PR has:
   
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] a release note entry in the PR description.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


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


[GitHub] [druid] imply-cheddar commented on pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on PR #14004:
URL: https://github.com/apache/druid/pull/14004#issuecomment-1588397747

   @abhishekagarwal87 once again, you've provided a great example!  Essentially, you went through the thought process that I'm hoping we can just do as a part of every exception message that we generate.
   
   In this case, making it ADMIN-focused and having more details about server names and stuff makes a lot of sense to me.  And the argument that someone who is trying to "hide" Druid should be able to siphon off this message makes a lot of sense to me too.  So, I think that I'm now convinced that the message in question *should* be targeting the ADMIN persona and specifically mention that you should check that the brokers are up and running.
   
   To me, this type of conversation, back-and-forth and your eventual conclusion is the whole goal of the adjustments to the error messages.  And, once we change that exception to be ADMIN focused, the code will include the fact that this is an ADMIN-focused message.  So any future person who comes across it can clearly see the intentions of who gets the message and why, if they feel the message needs updating, they can have their own internal dialog and motivate their feelings by either saying it should target a different persona OR it needs more details to target the current persona.
   
   So anyway, I think that yes, you are thinking along the same lines that I'm thinking at this point.  I'm all for adding documentation about whatever, but I'm unclear on where you think it would be added for the most impact.  Ultimately, this has to become part of the culture of the project rather than be a pure documentation thing, but I can document it anywhere you think it can help ;)


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


[GitHub] [druid] abhishekagarwal87 commented on pull request #14004: Errors take 3

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #14004:
URL: https://github.com/apache/druid/pull/14004#issuecomment-1587279726

   Now I am also wondering how we will pick a persona :) For example, in this case, I would like to tell the user that you really need to go and check the logs of the broker service. The action of checking error logs is something that an admin persona can do. But most people when they are setting up druid for the first time, are end-users as well as an admin. The error occurs directly in response to something that an end-user is doing. 
   
   I can return a user-persona-facing error saying that "Contact your cluster administrator to make sure that services are operational" but it's difficult for new users to figure out where to go from there. 
   
   So, in this case, maybe we return an error message meant for the admin persona even though this error is in user-initiated request-response path. We leave it to cluster admins if they want to filter messages that match admin persona, in case they are not exposing druid directly. For a new user setting up druid for first time, however they will get all these messages unfiltered. If you have something similar in mind, we should ideally document that developers should avoid adding potentially sensitive information in messages that are meant for user persona. 


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


[GitHub] [druid] clintropolis commented on a diff in pull request #14004: Errors take 3

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1231824195


##########
processing/src/main/java/org/apache/druid/error/README.md:
##########
@@ -0,0 +1,404 @@
+<!--
+  ~ 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.
+  -->
+
+WARNING WARNING
+TODO: this README has not been adjusted to align with the current code

Review Comment:
   is this still true? (maybe it should be refreshed to align with the current code before we merge?)
   
   also any possible way to add something like a 'tl;dr' to summarize the important parts? 😛 



##########
processing/src/main/java/org/apache/druid/error/ErrorResponse.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonValue;
+import org.apache.druid.query.QueryException;
+
+import javax.annotation.Nullable;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * A Response Object that represents an error to be returned over the wire.  This object carries legacy bits to
+ * deal with compatibility issues of converging the error responses from {@link QueryException}
+ * with the intended going-forward error responses from {@link DruidException}
+ * <p>
+ * The intent is that eventually {@link QueryException} is completely subsumed by
+ * {@link DruidException} in which case the legacy bits of this class can hopefully also be removed.
+ * <p>
+ * The intended long-term schema of output is an object that looks like
+ * <p>
+ * {
+ * "errorCode": `a code string`,
+ * "persona": USER | ADMIN | OPERATOR | DEVELOPER
+ * "category": DEFENSIVE | INVALID_INPUT | UNAUTHORIZED | CAPACITY_EXCEEDED | CANCELED | RUNTIME_FAILURE | TIMEOUT | UNSUPPORTED | UNCATEGORIZED
+ * "errorMessage": `a message for the intended audience`
+ * "context": `a map of extra context values that might be helpful`
+ * }
+ * <p>
+ * In the interim, there are extra fields that also end up included so that the wire-schema can also be interpretted
+ * and handled by clients that are built assuming they are looking at QueryExceptions.  These extra fields are
+ * <p>
+ * {
+ * "error": `an error code from QueryException` | "druidException"
+ * "errorClass": `the error class, as used by QueryException`
+ * "host": `the host that the exception occurred on, as used by QueryException`
+ * }
+ * <p>
+ * These 3 top-level fields are deprecated and will eventually disappear from API responses.  The values can, instead,
+ * be pulled from the context object of an "legacyQueryException" errorCode object.  The field names in the context
+ * object map as follows
+ * * "error" -> "legacyErrorCode"
+ * * "errorClass" -> "errorClass"
+ * * "host" -> "host"
+ */
+public class ErrorResponse
+{
+  @JsonCreator
+  public static ErrorResponse fromMap(Map<String, Object> map)
+  {
+    final DruidException.Failure failure;
+
+    final Object legacyErrorType = map.get("error");
+    if (!"druidException".equals(legacyErrorType)) {
+      // The non "druidException" errorCode field means that we are deserializing a legacy QueryException object rather
+      // than deserializing a DruidException.  So, we make a QueryException, map it to a DruidException and build
+      // our response from that DruidException.  This allows all code after us to only consider DruidException
+      // and helps aid the removal of QueryException.
+      failure = new QueryExceptionCompat(
+          new QueryException(
+              nullOrString(map.get("error")),
+              nullOrString(map.get("errorMessage")),
+              nullOrString(map.get("errorClass")),
+              nullOrString(map.get("host"))
+          )
+      );
+    } else {
+      failure = new DruidException.Failure(stringOrFailure(map, "errorCode"))
+      {
+        @Override
+        protected DruidException makeException(DruidException.DruidExceptionBuilder bob)
+        {
+          final DruidException retVal = bob.forPersona(DruidException.Persona.valueOf(stringOrFailure(map, "persona")))
+                                           .ofCategory(DruidException.Category.valueOf(stringOrFailure(
+                                               map,
+                                               "category"
+                                           )))
+                                           .build(stringOrFailure(map, "errorMessage"));
+
+          final Object context = map.get("context");
+          if (context instanceof Map) {
+            //noinspection unchecked
+            retVal.withContext((Map<String, String>) context);
+          }
+
+          return retVal;
+        }
+      };
+    }
+    return new ErrorResponse(DruidException.fromFailure(new DeserializedFailure(failure)));
+  }
+
+  private final DruidException underlyingException;
+
+  public ErrorResponse(DruidException underlyingException)
+  {
+    this.underlyingException = underlyingException;
+  }
+
+  @JsonValue
+  public Map<String, Object> getAsMap()
+  {
+    final LinkedHashMap<String, Object> retVal = new LinkedHashMap<>();
+
+    // This if statement is a compatibility layer to help bridge the time while we are introducing the DruidException.
+    // In a future release, QueryException should be completely eliminated, at which point we should also be
+    // able to eliminate this compatibility layer.
+    if (QueryExceptionCompat.ERROR_CODE.equals(underlyingException.getErrorCode())) {
+      retVal.put("error", underlyingException.getContextValue("legacyErrorCode"));
+      retVal.put("errorClass", underlyingException.getContextValue("errorClass"));
+      retVal.put("host", underlyingException.getContextValue("host"));
+    } else {
+      retVal.put("error", "druidException");
+    }
+
+    retVal.put("errorCode", underlyingException.getErrorCode());
+    retVal.put("persona", underlyingException.getTargetPersona().toString());
+    retVal.put("category", underlyingException.getCategory().toString());
+    retVal.put("errorMessage", underlyingException.getMessage());
+    retVal.put("context", underlyingException.getContext());
+
+    return retVal;
+  }
+
+  public DruidException getUnderlyingException()
+  {
+    return underlyingException;
+  }
+
+  @Nullable
+  private static String nullOrString(Object o)

Review Comment:
   there is a similar existing method, `Evals.asString` though its primarily used for processing data for expressions and query time stuff, so i don't feel super strongly about using it here or not



##########
processing/src/main/java/org/apache/druid/error/InvalidSqlInput.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+public class InvalidSqlInput extends InvalidInput

Review Comment:
   heh, that isn't really possible without also merging druid-server into processing, or alternatively splitting druid-sql operators and stuff into processing and putting the schema and higher level stuff that builds on broker stuff in druid-server (which might also be nice, but would make testing slightly more painful maybe?)



##########
server/src/test/java/org/apache/druid/server/QueryResourceTest.java:
##########
@@ -296,6 +302,79 @@ public void testGoodQueryWithQueryConfigOverrideDefault() throws IOException
     );
   }
 
+  @Test
+  public void testGoodQueryThrowsDruidExceptionFromLifecycleExecute() throws IOException
+  {
+    String overrideConfigKey = "priority";
+    String overrideConfigValue = "678";
+    DefaultQueryConfig overrideConfig = new DefaultQueryConfig(ImmutableMap.of(overrideConfigKey, overrideConfigValue));
+    queryResource = new QueryResource(
+        new QueryLifecycleFactory(
+            WAREHOUSE,
+            new QuerySegmentWalker()
+            {
+              @Override
+              public <T> QueryRunner<T> getQueryRunnerForIntervals(
+                  Query<T> query,
+                  Iterable<Interval> intervals
+              )
+              {
+                throw DruidException.forPersona(DruidException.Persona.OPERATOR)
+                                    .ofCategory(DruidException.Category.RUNTIME_FAILURE)
+                                    .build("failing for coverage!");
+              }
+
+              @Override
+              public <T> QueryRunner<T> getQueryRunnerForSegments(
+                  Query<T> query,
+                  Iterable<SegmentDescriptor> specs
+              )
+              {
+                throw new UnsupportedOperationException();
+              }
+            },
+            new DefaultGenericQueryMetricsFactory(),
+            new NoopServiceEmitter(),
+            testRequestLogger,
+            new AuthConfig(),
+            AuthTestUtils.TEST_AUTHORIZER_MAPPER,
+            Suppliers.ofInstance(overrideConfig)
+        ),
+        jsonMapper,
+        smileMapper,
+        queryScheduler,
+        new AuthConfig(),
+        null,
+        ResponseContextConfig.newConfig(true),
+        DRUID_NODE
+    );
+
+    expectPermissiveHappyPathAuth();
+
+    final Response response = expectSynchronousRequestFlow(SIMPLE_TIMESERIES_QUERY);
+    Assert.assertEquals(Status.INTERNAL_SERVER_ERROR.getStatusCode(), response.getStatus());
+
+    final ErrorResponse entity = (ErrorResponse) response.getEntity();

Review Comment:
   ah i guess this is the serde test i was looking for?



##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.

Review Comment:
   i think there is still a use case for them maybe for low level exceptions that don't have the full picture because they are missing some context to provide a meaningful exception at the surface. In these cases I expect the code area which does have the context will catch IAE/ISE/UOE etc and then extract the details and decorate with additional context such as column and table name, etc.
   
   That seems preferable to me than having to push all of this information down all of the way to the bottom, particularly in more 'hot' areas of the code since passing arguments isn't completely free. Though, I suspect some of these cases are probably better to use more specific exceptions dedicated to a purpose, _or_ I guess we could still throw `DruidException` but catch it at a higher level to decorate with the additional context, so I think we'll just have to see how it works out in practice.



##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java:
##########
@@ -307,4 +317,186 @@ public PlannerHook hook()
       return hook;
     }
   }
+
+  public static DruidException translateException(Exception e)
+  {
+    try {
+      throw e;
+    }

Review Comment:
   any reason not to just make all of these `instanceof` checks, especially since some of the catch still have `instanceof`?



##########
processing/src/test/java/org/apache/druid/error/ErrorResponseTest.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 com.google.common.collect.ImmutableMap;
+import org.apache.druid.matchers.DruidMatchers;
+import org.apache.druid.query.QueryTimeoutException;
+import org.hamcrest.Matcher;
+import org.hamcrest.MatcherAssert;
+import org.hamcrest.Matchers;
+import org.junit.Test;
+
+import java.util.Map;
+
+public class ErrorResponseTest

Review Comment:
   since ErrorResponse is an 'over the wire' thingo, should serde tests be here? (or did i miss them somewhere?)



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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1233475989


##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java:
##########
@@ -1673,7 +1659,7 @@ public void testGroupByWithComplexColumnThrowsUnsupportedException()
         .setSql("select unique_dim1 from foo2 group by unique_dim1")
         .setQueryContext(context)
         .setExpectedExecutionErrorMatcher(CoreMatchers.allOf(
-            CoreMatchers.instanceOf(UnsupportedSQLQueryException.class),
+            CoreMatchers.instanceOf(DruidException.class),

Review Comment:
   Technically various of the SQL exceptions get caught and used to tell the volcano planner that it generated a bad query and to keep looking.  None of those are converted to DruidException yet.  Also, most of the places that were throwing `QueryException` are still throwing `QueryException` because we need a release that has the `DruidException` in it before we can force those forward.
   
   So... really, any place that potentially has other uses just didn't get converted.  We will need to go back and review them with the understanding that we now have an exception whose sole purposes is to be delivered back to the user and decide if the current handling is good.



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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1226106653


##########
processing/src/test/java/org/apache/druid/matchers/DMatchers.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.matchers;
+
+import org.hamcrest.Matcher;
+
+import java.util.function.Function;
+
+public class DMatchers

Review Comment:
   Going away from the PR and coming back to it, I have no clue what it is either :P  Will look into it and update.



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


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #14004: Errors take 3

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1226115762


##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.
+ * <p>
+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception
+ * will make its way back to the user.  DruidException is always the answer to "how do I generate an error message and
+ * deliver it to the user"?
+ * <p>
+ * Every error consists of:
+ * <ul>
+ * <li>A target persona</li>
+ * <li>A categorization of the error</li>
+ * <li>An error code</li>
+ * <li>An error message</li>
+ * <li>A context (possibly empty)</li>
+ * </ul>
+ * <p>
+ * <p>
+ * The target persona indicates who the message is written for.  This is important for 2 reasons
+ * <ol>
+ *   <li>It identifies why the developer is creating the exception and who they believe can take action on it.
+ *   This context allows for code reviewers and other developers to evaluate the message with this context in mind</li>
+ *   <li>It can be used as a way to control which error messages should be routed where.  For example, a user-targetted
+ *   error message should be able to be exposed directly to the user, while an operator-targetted error message should
+ *   perhaps be routed to the operators of the system instead of the end user firing a query.</li>
+ * </ol>
+ * <p>
+ * The category indicates what kind of failure occurred.  This is leveraged to align response codes (e.g. HTTP response
+ * codes) for similar exception messages.
+ * <p>
+ * The error code is a code that indicates a grouping of error messages.  There is no forced structure around whether
+ * a specific error code can be reused for different problems or not.  That is, an error code like "adhoc" will get
+ * reused in many different places as it's the basic error code used whenever a DruidException is created in-line.  But,
+ * we might decide that a specific type of error should be identified explicitly by its error code and should only mean
+ * one thing, in which case that error code might only exist on a single error.
+ * <p>
+ * The error message is a message written targetting the target persona.  It should have values interpolated into it
+ * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information.
+ * <p>
+ * The context is a place to add extra information about the error that is not necessarily interpolated into the
+ * error message.  It's a way to carry extra information that might be useful to a developer, but not necessarily to
+ * the target persona.
+ *
+ * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}.
+ * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using
+ * forPersona, it will always be an "adhoc" error.
+ * <p>
+ * Additionally, DruidException is not intended to be directly serialized.  The intention is that something converts
+ * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that
+ * ErrorResponse is used for serialization.  DruidException carries a {@link #toErrorResponse()} method because there
+ * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that
+ * introduced DruidException.
+ */
+@NotThreadSafe
+public class DruidException extends RuntimeException
+{
+  /**
+   * Starts building an "adhoc" DruidException targetting the specific persona.
+   *
+   * @param persona the target persona of the exception message
+   * @return a builder that can be used to complete the creation of the DruidException
+   */
+  public static DruidExceptionBuilder forPersona(Persona persona)
+  {
+    return new DruidExceptionBuilder("adhoc").forPersona(persona);
+  }
+
+  /**
+   * Builds a DruidException using the provided Failure class.  The errorCode is determined by the
+   * specific Failure class being used and the Failure class is responsible for setting all other
+   * required fields of the DruidException
+   *
+   * @param failure failure implementation to use to build the DruidException
+   * @return DruidException instance built from the Failure instance provided
+   */
+  public static DruidException fromFailure(Failure failure)
+  {
+    return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode()));
+  }
+
+  private final Persona targetPersona;
+  private final Category category;
+  private final String errorCode;
+  protected final Map<String, String> context = new LinkedHashMap<>();
+
+  private DruidException(
+      Throwable cause,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      final String message
+  )
+  {
+    this(cause, errorCode, targetPersona, category, message, false);
+  }
+
+  private DruidException(
+      Throwable throwable,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      String message,
+      boolean deserialized
+  )
+  {
+    super(message, throwable, true, !deserialized);
+    this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
+    this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona");
+    this.category = Preconditions.checkNotNull(category, "category");
+  }
+
+  public DruidException withContext(String key, Object value)
+  {
+    context.put(key, value == null ? null : value.toString());
+    return this;
+  }
+
+  public DruidException withContext(Map<String, String> values)
+  {
+    this.context.putAll(values);
+    return this;
+  }
+
+  public Persona getTargetPersona()
+  {
+    return targetPersona;
+  }
+
+  public Category getCategory()
+  {
+    return category;
+  }
+
+  public String getErrorCode()
+  {
+    return errorCode;
+  }
+
+  public String getContextValue(String key)
+  {
+    return context.get(key);
+  }
+
+  public Map<String, String> getContext()
+  {
+    return context;
+  }
+
+  public int getStatusCode()
+  {
+    return category.getExpectedStatus();
+  }
+
+  /**
+   * Returns this DruidException as an ErrorResponse.  This method exists for compatibility with some older code
+   * paths that serialize out Exceptions directly using Jackson.  Instead of serializing a DruidException
+   * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be
+   * used to push across the wire.
+   * <p>
+   * As such, this method should be deleted in some future world.  Anyone wondering how to serialize and deserialize
+   * a DruidException should look at {@link ErrorResponse} and leverage that instead of this.
+   *
+   * @return an ErrorResponse
+   */
+  @JsonValue
+  public ErrorResponse toErrorResponse()
+  {
+    return new ErrorResponse(this);
+  }
+
+  /**
+   * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter
+   * with the message already on the DruidException.
+   *
+   * @param msg  Message to be prepended, can be a Java format string
+   * @param args Arguments to be passed to the message if it is a Java format string
+   * @return a new DruidException with prepended-message
+   */
+  public DruidException prependAndBuild(String msg, Object... args)
+  {
+    return new DruidException(
+        this,
+        errorCode,
+        targetPersona,
+        category,
+        StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage())
+    ).withContext(context);
+  }
+
+  /**
+   * The persona that the message on a DruidException is targetting
+   */
+  public enum Persona
+  {
+    /**
+     * Represents the end-user, a persona who is issuing queries to the Druid Query APIs
+     */
+    USER,
+    /**
+     * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query
+     * concepts without necessarily owning the infrastructure and operations of the cluster
+     */
+    ADMIN,
+    /**
+     * Represents a persona who actively owns and operates the cluster.  This persona is not assumed to understand
+     * Druid query concepts, but instead understand cluster operational concepts.
+     */
+    OPERATOR,
+    /**
+     * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase.
+     * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not
+     * possible to make a message that will make sense to a different persona.  Generally speaking, there is a hope
+     * that only DEFENSIVE error messages will target this persona.
+     */
+    DEVELOPER
+  }
+
+  /**
+   * Category of error.  The simplest way to describe this is that it exists as a classification of errors that
+   * enables us to identify the expected response code (e.g. HTTP status code) of a specific DruidException
+   */
+  public enum Category
+  {
+    /**
+     * Means that the exception is being created defensively, because we want to validate something but expect that
+     * it should never actually be hit.  Using this category is good to provide an indication to future reviewers and
+     * developers that the case being checked is not intended to actually be able to occur in the wild.
+     */
+    DEFENSIVE(500),
+    /**
+     * Means that the input provided was malformed in some way.  Generally speaking, it is hoped that errors of this
+     * category have messages written either targetting the USER or ADMIN personas as those are the general users
+     * of the APIs who could generate invalid inputs.
+     */
+    INVALID_INPUT(400),
+    /**
+     * Means that the error is a problem with authorization.
+     */
+    UNAUTHORIZED(401),
+    /**
+     * Means that some capacity limit was exceeded, this could be due to throttling or due to some system limit
+     */
+    CAPACITY_EXCEEDED(429),
+    /**
+     * Means that the query was canceled for some reason
+     */
+    CANCELED(500),
+    /**
+     * Indicates a server-side failure of some sort at runtime
+     */
+    RUNTIME_FAILURE(500),
+    /**
+     * A timeout happened
+     */
+    TIMEOUT(504),
+    /**
+     * Indicates some unsupported behavior was requested.  TODO
+     */
+    UNSUPPORTED(501),

Review Comment:
   so lets say someone issues a `UNION` query with de-duplicating while only supports `UNION ALL`. would you consider that as a `501` or `400`? Such planning errors are very common. I am concerned about the error code for such scenarios because a cluster admin might have set an alert on 5xx but not on 4xx. Since the former will indicate the server being bad while the latter indicates the user being bad. 
   
   



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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1226108500


##########
processing/src/main/java/org/apache/druid/error/InvalidSqlInput.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+public class InvalidSqlInput extends InvalidInput

Review Comment:
   Yes, it is weird that `sql` is not in `processing` (see what I did there? :P)



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


[GitHub] [druid] abhishekagarwal87 commented on pull request #14004: Errors take 3

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #14004:
URL: https://github.com/apache/druid/pull/14004#issuecomment-1588555414

   Re culture, I agree. I think we add these bits in the APIs javadocs, that should be more than enough. And then reviewers can just say, "The error message isn't so great. please read the javadocs for reference" 


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


[GitHub] [druid] clintropolis commented on pull request #14004: Errors take 3

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on PR #14004:
URL: https://github.com/apache/druid/pull/14004#issuecomment-1594190422

   also, should there be any web-console changes to go with the adjustments to error 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.

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


[GitHub] [druid] cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "cheddar (via GitHub)" <gi...@apache.org>.
cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1231885115


##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidPlanner.java:
##########
@@ -307,4 +317,186 @@ public PlannerHook hook()
       return hook;
     }
   }
+
+  public static DruidException translateException(Exception e)
+  {
+    try {
+      throw e;
+    }

Review Comment:
   not really, I think I started doing that and then stopped for some reason, I don't remember why though.



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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1226110310


##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.
+ * <p>
+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception
+ * will make its way back to the user.  DruidException is always the answer to "how do I generate an error message and
+ * deliver it to the user"?
+ * <p>
+ * Every error consists of:
+ * <ul>
+ * <li>A target persona</li>
+ * <li>A categorization of the error</li>
+ * <li>An error code</li>
+ * <li>An error message</li>
+ * <li>A context (possibly empty)</li>
+ * </ul>
+ * <p>
+ * <p>
+ * The target persona indicates who the message is written for.  This is important for 2 reasons
+ * <ol>
+ *   <li>It identifies why the developer is creating the exception and who they believe can take action on it.
+ *   This context allows for code reviewers and other developers to evaluate the message with this context in mind</li>
+ *   <li>It can be used as a way to control which error messages should be routed where.  For example, a user-targetted
+ *   error message should be able to be exposed directly to the user, while an operator-targetted error message should
+ *   perhaps be routed to the operators of the system instead of the end user firing a query.</li>
+ * </ol>
+ * <p>
+ * The category indicates what kind of failure occurred.  This is leveraged to align response codes (e.g. HTTP response
+ * codes) for similar exception messages.
+ * <p>
+ * The error code is a code that indicates a grouping of error messages.  There is no forced structure around whether
+ * a specific error code can be reused for different problems or not.  That is, an error code like "adhoc" will get
+ * reused in many different places as it's the basic error code used whenever a DruidException is created in-line.  But,
+ * we might decide that a specific type of error should be identified explicitly by its error code and should only mean
+ * one thing, in which case that error code might only exist on a single error.
+ * <p>
+ * The error message is a message written targetting the target persona.  It should have values interpolated into it
+ * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information.
+ * <p>
+ * The context is a place to add extra information about the error that is not necessarily interpolated into the
+ * error message.  It's a way to carry extra information that might be useful to a developer, but not necessarily to
+ * the target persona.
+ *
+ * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}.
+ * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using
+ * forPersona, it will always be an "adhoc" error.
+ * <p>
+ * Additionally, DruidException is not intended to be directly serialized.  The intention is that something converts
+ * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that
+ * ErrorResponse is used for serialization.  DruidException carries a {@link #toErrorResponse()} method because there
+ * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that
+ * introduced DruidException.
+ */
+@NotThreadSafe
+public class DruidException extends RuntimeException
+{
+  /**
+   * Starts building an "adhoc" DruidException targetting the specific persona.
+   *
+   * @param persona the target persona of the exception message
+   * @return a builder that can be used to complete the creation of the DruidException
+   */
+  public static DruidExceptionBuilder forPersona(Persona persona)
+  {
+    return new DruidExceptionBuilder("adhoc").forPersona(persona);
+  }
+
+  /**
+   * Builds a DruidException using the provided Failure class.  The errorCode is determined by the
+   * specific Failure class being used and the Failure class is responsible for setting all other
+   * required fields of the DruidException
+   *
+   * @param failure failure implementation to use to build the DruidException
+   * @return DruidException instance built from the Failure instance provided
+   */
+  public static DruidException fromFailure(Failure failure)
+  {
+    return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode()));
+  }
+
+  private final Persona targetPersona;
+  private final Category category;
+  private final String errorCode;
+  protected final Map<String, String> context = new LinkedHashMap<>();
+
+  private DruidException(
+      Throwable cause,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      final String message
+  )
+  {
+    this(cause, errorCode, targetPersona, category, message, false);
+  }
+
+  private DruidException(
+      Throwable throwable,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      String message,
+      boolean deserialized
+  )
+  {
+    super(message, throwable, true, !deserialized);
+    this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
+    this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona");
+    this.category = Preconditions.checkNotNull(category, "category");
+  }
+
+  public DruidException withContext(String key, Object value)
+  {
+    context.put(key, value == null ? null : value.toString());
+    return this;
+  }
+
+  public DruidException withContext(Map<String, String> values)
+  {
+    this.context.putAll(values);
+    return this;
+  }
+
+  public Persona getTargetPersona()
+  {
+    return targetPersona;
+  }
+
+  public Category getCategory()
+  {
+    return category;
+  }
+
+  public String getErrorCode()
+  {
+    return errorCode;
+  }
+
+  public String getContextValue(String key)
+  {
+    return context.get(key);
+  }
+
+  public Map<String, String> getContext()
+  {
+    return context;
+  }
+
+  public int getStatusCode()
+  {
+    return category.getExpectedStatus();
+  }
+
+  /**
+   * Returns this DruidException as an ErrorResponse.  This method exists for compatibility with some older code
+   * paths that serialize out Exceptions directly using Jackson.  Instead of serializing a DruidException
+   * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be
+   * used to push across the wire.
+   * <p>
+   * As such, this method should be deleted in some future world.  Anyone wondering how to serialize and deserialize
+   * a DruidException should look at {@link ErrorResponse} and leverage that instead of this.
+   *
+   * @return an ErrorResponse
+   */
+  @JsonValue
+  public ErrorResponse toErrorResponse()
+  {
+    return new ErrorResponse(this);
+  }
+
+  /**
+   * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter
+   * with the message already on the DruidException.
+   *
+   * @param msg  Message to be prepended, can be a Java format string
+   * @param args Arguments to be passed to the message if it is a Java format string
+   * @return a new DruidException with prepended-message
+   */
+  public DruidException prependAndBuild(String msg, Object... args)
+  {
+    return new DruidException(
+        this,
+        errorCode,
+        targetPersona,
+        category,
+        StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage())
+    ).withContext(context);
+  }
+
+  /**
+   * The persona that the message on a DruidException is targetting
+   */
+  public enum Persona
+  {
+    /**
+     * Represents the end-user, a persona who is issuing queries to the Druid Query APIs
+     */
+    USER,
+    /**
+     * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query
+     * concepts without necessarily owning the infrastructure and operations of the cluster
+     */
+    ADMIN,
+    /**
+     * Represents a persona who actively owns and operates the cluster.  This persona is not assumed to understand
+     * Druid query concepts, but instead understand cluster operational concepts.
+     */
+    OPERATOR,
+    /**
+     * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase.
+     * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not
+     * possible to make a message that will make sense to a different persona.  Generally speaking, there is a hope
+     * that only DEFENSIVE error messages will target this persona.
+     */
+    DEVELOPER
+  }
+
+  /**
+   * Category of error.  The simplest way to describe this is that it exists as a classification of errors that
+   * enables us to identify the expected response code (e.g. HTTP status code) of a specific DruidException
+   */
+  public enum Category
+  {
+    /**
+     * Means that the exception is being created defensively, because we want to validate something but expect that
+     * it should never actually be hit.  Using this category is good to provide an indication to future reviewers and
+     * developers that the case being checked is not intended to actually be able to occur in the wild.
+     */
+    DEFENSIVE(500),
+    /**
+     * Means that the input provided was malformed in some way.  Generally speaking, it is hoped that errors of this
+     * category have messages written either targetting the USER or ADMIN personas as those are the general users
+     * of the APIs who could generate invalid inputs.
+     */
+    INVALID_INPUT(400),
+    /**
+     * Means that the error is a problem with authorization.
+     */
+    UNAUTHORIZED(401),
+    /**
+     * Means that some capacity limit was exceeded, this could be due to throttling or due to some system limit
+     */
+    CAPACITY_EXCEEDED(429),
+    /**
+     * Means that the query was canceled for some reason
+     */
+    CANCELED(500),
+    /**
+     * Indicates a server-side failure of some sort at runtime
+     */
+    RUNTIME_FAILURE(500),
+    /**
+     * A timeout happened
+     */
+    TIMEOUT(504),
+    /**
+     * Indicates some unsupported behavior was requested.  TODO
+     */
+    UNSUPPORTED(501),

Review Comment:
   501 == "Not Implemented" i.e. it's using functionality that could work but hasn't been implemented yet.  You could argue that it's user-error for using something that's not implemented yet.  Or you could argue that it's the server's fault for not implementing the thing.  
   
   IIRC, this is pretty seldom used and it's mostly used in SQL planning stuffs for things that *should* exist as far as SQL is concerned, but don't yet.



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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1226108265


##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java:
##########
@@ -125,12 +126,12 @@ public void reduce(
               // the query can execute.
               double exprResultDouble = exprResult.asDouble();
               if (Double.isNaN(exprResultDouble) || Double.isInfinite(exprResultDouble)) {
-                String expression = druidExpression.getExpression();
-                throw new UnsupportedSQLQueryException("'%s' evaluates to '%s' that is not supported in SQL. You can either cast the expression as BIGINT ('CAST(%s as BIGINT)') or VARCHAR ('CAST(%s as VARCHAR)') or change the expression itself",
-                    expression,
-                    Double.toString(exprResultDouble),
-                    expression,
-                    expression);
+                throw InvalidSqlInput.exception(
+                        "Expression [%s] evaluates to an unsupported value [%s], expected something that"
+                        + " can be a Double.  Consider casting with 'CAST(<col> AS BIGINT)'",
+                        druidExpression.getExpression(),
+                        exprResultDouble
+                    );

Review Comment:
   I took it as the expression has a return type and that return type isn't matching the input of another expression or something like that.  Which is why it had the suggestion to cast...  But, I was mostly just keeping what was in existence around rather than creating anything new.



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


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #14004: Errors take 3

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1154227559


##########
sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java:
##########
@@ -249,60 +254,66 @@
   protected AbstractAvaticaHandler getAvaticaHandler(final DruidMeta druidMeta)
   {
     return new DruidAvaticaJsonHandler(
-            druidMeta,
-            new DruidNode("dummy", "dummy", false, 1, null, true, false),
-            new AvaticaMonitor()
+        druidMeta,
+        new DruidNode("dummy", "dummy", false, 1, null, true, false),
+        new AvaticaMonitor()
     );
   }
 
   @Before
   public void setUp() throws Exception
   {
-    walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder());
     final DruidSchemaCatalog rootSchema = makeRootSchema();
     testRequestLogger = new TestRequestLogger();
 
     injector = new CoreInjectorBuilder(new StartupInjectorBuilder().build())
-        .addModule(binder -> {
-            binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test");
-            binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
-            binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1);
-            binder.bind(AuthenticatorMapper.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_MAPPER);
-            binder.bind(AuthorizerMapper.class).toInstance(CalciteTests.TEST_AUTHORIZER_MAPPER);
-            binder.bind(Escalator.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR);
-            binder.bind(RequestLogger.class).toInstance(testRequestLogger);
-            binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema);
-            for (NamedSchema schema : rootSchema.getNamedSchemas().values()) {
-              Multibinder.newSetBinder(binder, NamedSchema.class).addBinding().toInstance(schema);
+        .addModule(
+            binder -> {
+              binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test");
+              binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0);
+              binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1);
+              binder.bind(AuthenticatorMapper.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_MAPPER);
+              binder.bind(AuthorizerMapper.class).toInstance(CalciteTests.TEST_AUTHORIZER_MAPPER);
+              binder.bind(Escalator.class).toInstance(CalciteTests.TEST_AUTHENTICATOR_ESCALATOR);
+              binder.bind(RequestLogger.class).toInstance(testRequestLogger);
+              binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema);
+              for (NamedSchema schema : rootSchema.getNamedSchemas().values()) {
+                Multibinder.newSetBinder(binder, NamedSchema.class).addBinding().toInstance(schema);
+              }
+              binder.bind(QueryLifecycleFactory.class)
+                    .toInstance(CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate));
+              binder.bind(DruidOperatorTable.class).toInstance(operatorTable);
+              binder.bind(ExprMacroTable.class).toInstance(macroTable);
+              binder.bind(PlannerConfig.class).toInstance(plannerConfig);
+              binder.bind(String.class)
+                    .annotatedWith(DruidSchemaName.class)
+                    .toInstance(CalciteTests.DRUID_SCHEMA_NAME);
+              binder.bind(AvaticaServerConfig.class).toInstance(AVATICA_CONFIG);
+              binder.bind(ServiceEmitter.class).to(NoopServiceEmitter.class);
+              binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class);
+              binder.bind(QueryScheduler.class)
+                    .toProvider(QuerySchedulerProvider.class)
+                    .in(LazySingleton.class);
+              binder.install(new SqlModule.SqlStatementFactoryModule());
+              binder.bind(new TypeLiteral<Supplier<DefaultQueryConfig>>()
+              {
+              }).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of())));
+              binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of()));
+              binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper());
+              binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER);
             }
-            binder.bind(QueryLifecycleFactory.class)
-                  .toInstance(CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate));
-            binder.bind(DruidOperatorTable.class).toInstance(operatorTable);
-            binder.bind(ExprMacroTable.class).toInstance(macroTable);
-            binder.bind(PlannerConfig.class).toInstance(plannerConfig);
-            binder.bind(String.class)
-                  .annotatedWith(DruidSchemaName.class)
-                  .toInstance(CalciteTests.DRUID_SCHEMA_NAME);
-            binder.bind(AvaticaServerConfig.class).toInstance(AVATICA_CONFIG);
-            binder.bind(ServiceEmitter.class).to(NoopServiceEmitter.class);
-            binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class);
-            binder.bind(QueryScheduler.class)
-                  .toProvider(QuerySchedulerProvider.class)
-                  .in(LazySingleton.class);
-            binder.install(new SqlModule.SqlStatementFactoryModule());
-            binder.bind(new TypeLiteral<Supplier<DefaultQueryConfig>>(){}).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of())));
-            binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of()));
-            binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper());
-            binder.bind(CatalogResolver.class).toInstance(CatalogResolver.NULL_RESOLVER);
-          }
-         )
+        )
         .build();
 
     DruidMeta druidMeta = injector.getInstance(DruidMeta.class);
     server = new ServerWrapper(druidMeta);
     client = server.getUserConnection();
     superuserClient = server.getConnection(CalciteTests.TEST_SUPERUSER_NAME, "druid");
-    clientNoTrailingSlash = DriverManager.getConnection(StringUtils.maybeRemoveTrailingSlash(server.url), CalciteTests.TEST_SUPERUSER_NAME, "druid");
+    clientNoTrailingSlash = DriverManager.getConnection(
+        StringUtils.maybeRemoveTrailingSlash(server.url),
+        CalciteTests.TEST_SUPERUSER_NAME,
+        "druid"

Review Comment:
   ## Hard-coded credential in API call
   
   Hard-coded value flows to [sensitive API call](1).
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4562)



##########
server/src/main/java/org/apache/druid/server/QueryResultPusher.java:
##########
@@ -197,58 +208,47 @@
   @Nullable
   private Response handleQueryException(ResultsWriter resultsWriter, QueryException e)
   {
-    if (accumulator != null && accumulator.isInitialized()) {
-      // We already started sending a response when we got the error message.  In this case we just give up
-      // and hope that the partial stream generates a meaningful failure message for our client.  We could consider
-      // also throwing the exception body into the response to make it easier for the client to choke if it manages
-      // to parse a meaningful object out, but that's potentially an API change so we leave that as an exercise for
-      // the future.
+    return handleDruidException(resultsWriter, DruidException.fromFailure(new QueryExceptionCompat(e)));
+  }
 
+  private Response handleDruidException(ResultsWriter resultsWriter, DruidException e)
+  {
+    if (resultsWriter != null) {
       resultsWriter.recordFailure(e);
-
-      // This case is always a failure because the error happened mid-stream of sending results back.  Therefore,
-      // we do not believe that the response stream was actually usable
       counter.incrementFailed();
-      return null;
+
+      if (accumulator != null && accumulator.isInitialized()) {
+        // We already started sending a response when we got the error message.  In this case we just give up
+        // and hope that the partial stream generates a meaningful failure message for our client.  We could consider
+        // also throwing the exception body into the response to make it easier for the client to choke if it manages
+        // to parse a meaningful object out, but that's potentially an API change so we leave that as an exercise for
+        // the future.
+        return null;
+      }
     }
 
-    final QueryException.FailType failType = e.getFailType();
-    switch (failType) {
-      case USER_ERROR:
+    switch (e.getCategory()) {

Review Comment:
   ## Missing enum case in switch
   
   Switch statement does not have a case for [DEFENSIVE](1).
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4508)



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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1228897694


##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.
+ * <p>
+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception
+ * will make its way back to the user.  DruidException is always the answer to "how do I generate an error message and
+ * deliver it to the user"?
+ * <p>
+ * Every error consists of:
+ * <ul>
+ * <li>A target persona</li>
+ * <li>A categorization of the error</li>
+ * <li>An error code</li>
+ * <li>An error message</li>
+ * <li>A context (possibly empty)</li>
+ * </ul>
+ * <p>
+ * <p>
+ * The target persona indicates who the message is written for.  This is important for 2 reasons
+ * <ol>
+ *   <li>It identifies why the developer is creating the exception and who they believe can take action on it.
+ *   This context allows for code reviewers and other developers to evaluate the message with this context in mind</li>
+ *   <li>It can be used as a way to control which error messages should be routed where.  For example, a user-targetted
+ *   error message should be able to be exposed directly to the user, while an operator-targetted error message should
+ *   perhaps be routed to the operators of the system instead of the end user firing a query.</li>
+ * </ol>
+ * <p>
+ * The category indicates what kind of failure occurred.  This is leveraged to align response codes (e.g. HTTP response
+ * codes) for similar exception messages.
+ * <p>
+ * The error code is a code that indicates a grouping of error messages.  There is no forced structure around whether
+ * a specific error code can be reused for different problems or not.  That is, an error code like "adhoc" will get
+ * reused in many different places as it's the basic error code used whenever a DruidException is created in-line.  But,
+ * we might decide that a specific type of error should be identified explicitly by its error code and should only mean
+ * one thing, in which case that error code might only exist on a single error.
+ * <p>
+ * The error message is a message written targetting the target persona.  It should have values interpolated into it
+ * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information.
+ * <p>
+ * The context is a place to add extra information about the error that is not necessarily interpolated into the
+ * error message.  It's a way to carry extra information that might be useful to a developer, but not necessarily to
+ * the target persona.
+ *
+ * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}.
+ * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using
+ * forPersona, it will always be an "adhoc" error.
+ * <p>
+ * Additionally, DruidException is not intended to be directly serialized.  The intention is that something converts
+ * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that
+ * ErrorResponse is used for serialization.  DruidException carries a {@link #toErrorResponse()} method because there
+ * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that
+ * introduced DruidException.
+ */
+@NotThreadSafe
+public class DruidException extends RuntimeException
+{
+  /**
+   * Starts building an "adhoc" DruidException targetting the specific persona.
+   *
+   * @param persona the target persona of the exception message
+   * @return a builder that can be used to complete the creation of the DruidException
+   */
+  public static DruidExceptionBuilder forPersona(Persona persona)
+  {
+    return new DruidExceptionBuilder("adhoc").forPersona(persona);
+  }
+
+  /**
+   * Builds a DruidException using the provided Failure class.  The errorCode is determined by the
+   * specific Failure class being used and the Failure class is responsible for setting all other
+   * required fields of the DruidException
+   *
+   * @param failure failure implementation to use to build the DruidException
+   * @return DruidException instance built from the Failure instance provided
+   */
+  public static DruidException fromFailure(Failure failure)
+  {
+    return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode()));
+  }
+
+  private final Persona targetPersona;
+  private final Category category;
+  private final String errorCode;
+  protected final Map<String, String> context = new LinkedHashMap<>();
+
+  private DruidException(
+      Throwable cause,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      final String message
+  )
+  {
+    this(cause, errorCode, targetPersona, category, message, false);
+  }
+
+  private DruidException(
+      Throwable throwable,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      String message,
+      boolean deserialized
+  )
+  {
+    super(message, throwable, true, !deserialized);
+    this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
+    this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona");
+    this.category = Preconditions.checkNotNull(category, "category");
+  }
+
+  public DruidException withContext(String key, Object value)
+  {
+    context.put(key, value == null ? null : value.toString());
+    return this;
+  }
+
+  public DruidException withContext(Map<String, String> values)
+  {
+    this.context.putAll(values);
+    return this;
+  }
+
+  public Persona getTargetPersona()
+  {
+    return targetPersona;
+  }
+
+  public Category getCategory()
+  {
+    return category;
+  }
+
+  public String getErrorCode()
+  {
+    return errorCode;
+  }
+
+  public String getContextValue(String key)
+  {
+    return context.get(key);
+  }
+
+  public Map<String, String> getContext()
+  {
+    return context;
+  }
+
+  public int getStatusCode()
+  {
+    return category.getExpectedStatus();
+  }
+
+  /**
+   * Returns this DruidException as an ErrorResponse.  This method exists for compatibility with some older code
+   * paths that serialize out Exceptions directly using Jackson.  Instead of serializing a DruidException
+   * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be
+   * used to push across the wire.
+   * <p>
+   * As such, this method should be deleted in some future world.  Anyone wondering how to serialize and deserialize
+   * a DruidException should look at {@link ErrorResponse} and leverage that instead of this.
+   *
+   * @return an ErrorResponse
+   */
+  @JsonValue
+  public ErrorResponse toErrorResponse()
+  {
+    return new ErrorResponse(this);
+  }
+
+  /**
+   * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter
+   * with the message already on the DruidException.
+   *
+   * @param msg  Message to be prepended, can be a Java format string
+   * @param args Arguments to be passed to the message if it is a Java format string
+   * @return a new DruidException with prepended-message
+   */
+  public DruidException prependAndBuild(String msg, Object... args)
+  {
+    return new DruidException(
+        this,
+        errorCode,
+        targetPersona,
+        category,
+        StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage())
+    ).withContext(context);
+  }
+
+  /**
+   * The persona that the message on a DruidException is targetting
+   */
+  public enum Persona
+  {
+    /**
+     * Represents the end-user, a persona who is issuing queries to the Druid Query APIs
+     */
+    USER,
+    /**
+     * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query
+     * concepts without necessarily owning the infrastructure and operations of the cluster
+     */
+    ADMIN,
+    /**
+     * Represents a persona who actively owns and operates the cluster.  This persona is not assumed to understand
+     * Druid query concepts, but instead understand cluster operational concepts.
+     */
+    OPERATOR,
+    /**
+     * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase.
+     * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not
+     * possible to make a message that will make sense to a different persona.  Generally speaking, there is a hope
+     * that only DEFENSIVE error messages will target this persona.
+     */
+    DEVELOPER
+  }
+
+  /**
+   * Category of error.  The simplest way to describe this is that it exists as a classification of errors that
+   * enables us to identify the expected response code (e.g. HTTP status code) of a specific DruidException
+   */
+  public enum Category
+  {
+    /**
+     * Means that the exception is being created defensively, because we want to validate something but expect that
+     * it should never actually be hit.  Using this category is good to provide an indication to future reviewers and
+     * developers that the case being checked is not intended to actually be able to occur in the wild.
+     */
+    DEFENSIVE(500),
+    /**
+     * Means that the input provided was malformed in some way.  Generally speaking, it is hoped that errors of this
+     * category have messages written either targetting the USER or ADMIN personas as those are the general users
+     * of the APIs who could generate invalid inputs.
+     */
+    INVALID_INPUT(400),
+    /**
+     * Means that the error is a problem with authorization.
+     */
+    UNAUTHORIZED(401),
+    /**
+     * Means that some capacity limit was exceeded, this could be due to throttling or due to some system limit
+     */
+    CAPACITY_EXCEEDED(429),
+    /**
+     * Means that the query was canceled for some reason
+     */
+    CANCELED(500),
+    /**
+     * Indicates a server-side failure of some sort at runtime
+     */
+    RUNTIME_FAILURE(500),
+    /**
+     * A timeout happened
+     */
+    TIMEOUT(504),
+    /**
+     * Indicates some unsupported behavior was requested.  TODO
+     */
+    UNSUPPORTED(501),

Review Comment:
   TBH, I'm fine with both.  The 501 code is actually just a hold-over from the QueryException rather than something I'm actually introducing.  If anything, reviewing this change has caused it to be more clear that we are returning 501s in some instances, leading to a question of "WTF", why?  Which I think is a totally valid question ;).
   
   Fwiw, I have a TODO in there because this is something I wanted to revisit myself.  At this point, the only usages of this in the code in this PR are
   
   https://github.com/apache/druid/pull/14004/files#diff-483e089932b8faa5ca9669607eab7fcba883d27e2899a7c9fba2f37afb9de97bR599-R602
   
   Which is what happens when we get a query planning failure and no message at all.  I don't know if/when this actually happens, so I'm unclear on whether it's actually meaningful.  That said, with the switch to the `DruidException`, we should be able to eliminate the whole `plannerContext().getPlanningError()` thing that is the source of potentially not having a message (and also is the source of red-herring error messages more often than not), which should, in turn, eliminate the need for this category.  Regardless, it needs to exist to maintain what QueryException is doing...
   
   



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


[GitHub] [druid] imply-cheddar commented on pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on PR #14004:
URL: https://github.com/apache/druid/pull/14004#issuecomment-1592462828

   Okay, had a clean bill. Aside from some static checks.  Going to push those up.  @abhishekagarwal87 @clintropolis I'd love to get this merged today or tomorrow if possible, please do what you need to do to feel comfortable approving.
   
   <img width="840" alt="image" src="https://github.com/apache/druid/assets/86940447/4e75deea-4fe6-47a5-8f7f-73b69888bdec">
   


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


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #14004: Errors take 3

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1165058756


##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.
+ * <p>
+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception
+ * will make its way back to the user.  DruidException is always the answer to "how do I generate an error message and
+ * deliver it to the user"?
+ * <p>
+ * Every error consists of:
+ * <ul>
+ * <li>A target persona</li>
+ * <li>A categorization of the error</li>
+ * <li>An error code</li>
+ * <li>An error message</li>
+ * <li>A context (possibly empty)</li>
+ * </ul>
+ * <p>
+ * <p>
+ * The target persona indicates who the message is written for.  This is important for 2 reasons
+ * <ol>
+ *   <li>It identifies why the developer is creating the exception and who they believe can take action on it.
+ *   This context allows for code reviewers and other developers to evaluate the message with this context in mind</li>
+ *   <li>It can be used as a way to control which error messages should be routed where.  For example, a user-targetted
+ *   error message should be able to be exposed directly to the user, while an operator-targetted error message should
+ *   perhaps be routed to the operators of the system instead of the end user firing a query.</li>
+ * </ol>
+ * <p>
+ * The category indicates what kind of failure occurred.  This is leveraged to align response codes (e.g. HTTP response
+ * codes) for similar exception messages.
+ * <p>
+ * The error code is a code that indicates a grouping of error messages.  There is no forced structure around whether
+ * a specific error code can be reused for different problems or not.  That is, an error code like "adhoc" will get
+ * reused in many different places as it's the basic error code used whenever a DruidException is created in-line.  But,
+ * we might decide that a specific type of error should be identified explicitly by its error code and should only mean
+ * one thing, in which case that error code might only exist on a single error.
+ * <p>
+ * The error message is a message written targetting the target persona.  It should have values interpolated into it
+ * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information.
+ * <p>
+ * The context is a place to add extra information about the error that is not necessarily interpolated into the
+ * error message.  It's a way to carry extra information that might be useful to a developer, but not necessarily to
+ * the target persona.
+ *
+ * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}.
+ * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using
+ * forPersona, it will always be an "adhoc" error.
+ * <p>
+ * Additionally, DruidException is not intended to be directly serialized.  The intention is that something converts
+ * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that
+ * ErrorResponse is used for serialization.  DruidException carries a {@link #toErrorResponse()} method because there
+ * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that
+ * introduced DruidException.
+ */
+@NotThreadSafe
+public class DruidException extends RuntimeException
+{
+  /**
+   * Starts building an "adhoc" DruidException targetting the specific persona.
+   *
+   * @param persona the target persona of the exception message
+   * @return a builder that can be used to complete the creation of the DruidException
+   */
+  public static DruidExceptionBuilder forPersona(Persona persona)
+  {
+    return new DruidExceptionBuilder("adhoc").forPersona(persona);
+  }
+
+  /**
+   * Builds a DruidException using the provided Failure class.  The errorCode is determined by the
+   * specific Failure class being used and the Failure class is responsible for setting all other
+   * required fields of the DruidException
+   *
+   * @param failure failure implementation to use to build the DruidException
+   * @return DruidException instance built from the Failure instance provided
+   */
+  public static DruidException fromFailure(Failure failure)
+  {
+    return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode()));
+  }
+
+  private final Persona targetPersona;
+  private final Category category;
+  private final String errorCode;
+  protected final Map<String, String> context = new LinkedHashMap<>();
+
+  private DruidException(
+      Throwable cause,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      final String message
+  )
+  {
+    this(cause, errorCode, targetPersona, category, message, false);
+  }
+
+  private DruidException(
+      Throwable throwable,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      String message,
+      boolean deserialized
+  )
+  {
+    super(message, throwable, true, !deserialized);
+    this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
+    this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona");
+    this.category = Preconditions.checkNotNull(category, "category");
+  }
+
+  public DruidException withContext(String key, Object value)
+  {
+    context.put(key, value == null ? null : value.toString());
+    return this;
+  }
+
+  public DruidException withContext(Map<String, String> values)
+  {
+    this.context.putAll(values);
+    return this;
+  }
+
+  public Persona getTargetPersona()
+  {
+    return targetPersona;
+  }
+
+  public Category getCategory()
+  {
+    return category;
+  }
+
+  public String getErrorCode()
+  {
+    return errorCode;
+  }
+
+  public String getContextValue(String key)
+  {
+    return context.get(key);
+  }
+
+  public Map<String, String> getContext()
+  {
+    return context;
+  }
+
+  public int getStatusCode()
+  {
+    return category.getExpectedStatus();
+  }
+
+  /**
+   * Returns this DruidException as an ErrorResponse.  This method exists for compatibility with some older code
+   * paths that serialize out Exceptions directly using Jackson.  Instead of serializing a DruidException
+   * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be
+   * used to push across the wire.
+   * <p>
+   * As such, this method should be deleted in some future world.  Anyone wondering how to serialize and deserialize
+   * a DruidException should look at {@link ErrorResponse} and leverage that instead of this.
+   *
+   * @return an ErrorResponse
+   */
+  @JsonValue
+  public ErrorResponse toErrorResponse()
+  {
+    return new ErrorResponse(this);
+  }
+
+  /**
+   * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter
+   * with the message already on the DruidException.
+   *
+   * @param msg  Message to be prepended, can be a Java format string
+   * @param args Arguments to be passed to the message if it is a Java format string
+   * @return a new DruidException with prepended-message
+   */
+  public DruidException prependAndBuild(String msg, Object... args)
+  {
+    return new DruidException(
+        this,
+        errorCode,
+        targetPersona,
+        category,
+        StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage())
+    ).withContext(context);
+  }
+
+  /**
+   * The persona that the message on a DruidException is targetting
+   */
+  public enum Persona
+  {
+    /**
+     * Represents the end-user, a persona who is issuing queries to the Druid Query APIs
+     */
+    USER,
+    /**
+     * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query
+     * concepts without necessarily owning the infrastructure and operations of the cluster
+     */
+    ADMIN,
+    /**
+     * Represents a persona who actively owns and operates the cluster.  This persona is not assumed to understand
+     * Druid query concepts, but instead understand cluster operational concepts.
+     */
+    OPERATOR,
+    /**
+     * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase.
+     * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not
+     * possible to make a message that will make sense to a different persona.  Generally speaking, there is a hope
+     * that only DEFENSIVE error messages will target this persona.
+     */
+    DEVELOPER
+  }
+
+  /**
+   * Category of error.  The simplest way to describe this is that it exists as a classification of errors that
+   * enables us to identify the expected response code (e.g. HTTP status code) of a specific DruidException
+   */
+  public enum Category
+  {
+    /**
+     * Means that the exception is being created defensively, because we want to validate something but expect that
+     * it should never actually be hit.  Using this category is good to provide an indication to future reviewers and
+     * developers that the case being checked is not intended to actually be able to occur in the wild.
+     */
+    DEFENSIVE(500),
+    /**
+     * Means that the input provided was malformed in some way.  Generally speaking, it is hoped that errors of this
+     * category have messages written either targetting the USER or ADMIN personas as those are the general users
+     * of the APIs who could generate invalid inputs.
+     */
+    INVALID_INPUT(400),
+    /**
+     * Means that the error is a problem with authorization.
+     */
+    UNAUTHORIZED(401),
+    /**
+     * Means that some capacity limit was exceeded, this could be due to throttling or due to some system limit
+     */
+    CAPACITY_EXCEEDED(429),
+    /**
+     * Means that the query was canceled for some reason
+     */
+    CANCELED(500),
+    /**
+     * Indicates a server-side failure of some sort at runtime
+     */
+    RUNTIME_FAILURE(500),
+    /**
+     * A timeout happened
+     */
+    TIMEOUT(504),
+    /**
+     * Indicates some unsupported behavior was requested.  TODO
+     */
+    UNSUPPORTED(501),

Review Comment:
   why is UNSUPPORTED a 501? shouldn't this be a 400 (Bad request)



##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.
+ * <p>
+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception
+ * will make its way back to the user.  DruidException is always the answer to "how do I generate an error message and
+ * deliver it to the user"?
+ * <p>
+ * Every error consists of:
+ * <ul>
+ * <li>A target persona</li>
+ * <li>A categorization of the error</li>
+ * <li>An error code</li>
+ * <li>An error message</li>
+ * <li>A context (possibly empty)</li>
+ * </ul>
+ * <p>
+ * <p>
+ * The target persona indicates who the message is written for.  This is important for 2 reasons
+ * <ol>
+ *   <li>It identifies why the developer is creating the exception and who they believe can take action on it.
+ *   This context allows for code reviewers and other developers to evaluate the message with this context in mind</li>
+ *   <li>It can be used as a way to control which error messages should be routed where.  For example, a user-targetted
+ *   error message should be able to be exposed directly to the user, while an operator-targetted error message should
+ *   perhaps be routed to the operators of the system instead of the end user firing a query.</li>
+ * </ol>
+ * <p>
+ * The category indicates what kind of failure occurred.  This is leveraged to align response codes (e.g. HTTP response
+ * codes) for similar exception messages.
+ * <p>
+ * The error code is a code that indicates a grouping of error messages.  There is no forced structure around whether
+ * a specific error code can be reused for different problems or not.  That is, an error code like "adhoc" will get
+ * reused in many different places as it's the basic error code used whenever a DruidException is created in-line.  But,
+ * we might decide that a specific type of error should be identified explicitly by its error code and should only mean
+ * one thing, in which case that error code might only exist on a single error.
+ * <p>
+ * The error message is a message written targetting the target persona.  It should have values interpolated into it
+ * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information.
+ * <p>
+ * The context is a place to add extra information about the error that is not necessarily interpolated into the
+ * error message.  It's a way to carry extra information that might be useful to a developer, but not necessarily to
+ * the target persona.
+ *
+ * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}.
+ * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using
+ * forPersona, it will always be an "adhoc" error.
+ * <p>
+ * Additionally, DruidException is not intended to be directly serialized.  The intention is that something converts
+ * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that
+ * ErrorResponse is used for serialization.  DruidException carries a {@link #toErrorResponse()} method because there
+ * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that
+ * introduced DruidException.
+ */
+@NotThreadSafe
+public class DruidException extends RuntimeException
+{
+  /**
+   * Starts building an "adhoc" DruidException targetting the specific persona.
+   *
+   * @param persona the target persona of the exception message
+   * @return a builder that can be used to complete the creation of the DruidException
+   */
+  public static DruidExceptionBuilder forPersona(Persona persona)
+  {
+    return new DruidExceptionBuilder("adhoc").forPersona(persona);
+  }
+
+  /**
+   * Builds a DruidException using the provided Failure class.  The errorCode is determined by the
+   * specific Failure class being used and the Failure class is responsible for setting all other
+   * required fields of the DruidException
+   *
+   * @param failure failure implementation to use to build the DruidException
+   * @return DruidException instance built from the Failure instance provided
+   */
+  public static DruidException fromFailure(Failure failure)
+  {
+    return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode()));
+  }
+
+  private final Persona targetPersona;
+  private final Category category;
+  private final String errorCode;
+  protected final Map<String, String> context = new LinkedHashMap<>();
+
+  private DruidException(
+      Throwable cause,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      final String message
+  )
+  {
+    this(cause, errorCode, targetPersona, category, message, false);
+  }
+
+  private DruidException(
+      Throwable throwable,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      String message,
+      boolean deserialized
+  )
+  {
+    super(message, throwable, true, !deserialized);
+    this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
+    this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona");
+    this.category = Preconditions.checkNotNull(category, "category");
+  }
+
+  public DruidException withContext(String key, Object value)
+  {
+    context.put(key, value == null ? null : value.toString());
+    return this;
+  }
+
+  public DruidException withContext(Map<String, String> values)
+  {
+    this.context.putAll(values);
+    return this;
+  }
+
+  public Persona getTargetPersona()
+  {
+    return targetPersona;
+  }
+
+  public Category getCategory()
+  {
+    return category;
+  }
+
+  public String getErrorCode()
+  {
+    return errorCode;
+  }
+
+  public String getContextValue(String key)
+  {
+    return context.get(key);
+  }
+
+  public Map<String, String> getContext()
+  {
+    return context;
+  }
+
+  public int getStatusCode()
+  {
+    return category.getExpectedStatus();
+  }
+
+  /**
+   * Returns this DruidException as an ErrorResponse.  This method exists for compatibility with some older code
+   * paths that serialize out Exceptions directly using Jackson.  Instead of serializing a DruidException
+   * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be
+   * used to push across the wire.
+   * <p>
+   * As such, this method should be deleted in some future world.  Anyone wondering how to serialize and deserialize
+   * a DruidException should look at {@link ErrorResponse} and leverage that instead of this.
+   *
+   * @return an ErrorResponse
+   */
+  @JsonValue
+  public ErrorResponse toErrorResponse()
+  {
+    return new ErrorResponse(this);
+  }
+
+  /**
+   * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter
+   * with the message already on the DruidException.
+   *
+   * @param msg  Message to be prepended, can be a Java format string
+   * @param args Arguments to be passed to the message if it is a Java format string
+   * @return a new DruidException with prepended-message
+   */
+  public DruidException prependAndBuild(String msg, Object... args)
+  {
+    return new DruidException(
+        this,
+        errorCode,
+        targetPersona,
+        category,
+        StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage())
+    ).withContext(context);
+  }
+
+  /**
+   * The persona that the message on a DruidException is targetting
+   */
+  public enum Persona
+  {
+    /**
+     * Represents the end-user, a persona who is issuing queries to the Druid Query APIs
+     */
+    USER,
+    /**
+     * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query
+     * concepts without necessarily owning the infrastructure and operations of the cluster
+     */
+    ADMIN,
+    /**
+     * Represents a persona who actively owns and operates the cluster.  This persona is not assumed to understand
+     * Druid query concepts, but instead understand cluster operational concepts.
+     */
+    OPERATOR,
+    /**
+     * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase.
+     * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not
+     * possible to make a message that will make sense to a different persona.  Generally speaking, there is a hope
+     * that only DEFENSIVE error messages will target this persona.
+     */
+    DEVELOPER

Review Comment:
   ok. this is where I assumed we can have IAE or ISE thrown. Would there be equivalents of IAE, ISE that are implementation of DruidException? 



##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.

Review Comment:
   I think that ISE, IAE will be thrown in places where the exception is likely due to a code bug as opposed to something that a user did. For example, in case one comes across an enum that is unknown or a type that is unknown. Maybe we have a different superclass for those exceptions. That superclass should encourage developers to include as much diagnostic information in the exception message as much as possible. 
   
   IAE, ISE --> 5xx
   DruidException -> 4xx



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


[GitHub] [druid] imply-cheddar commented on pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on PR #14004:
URL: https://github.com/apache/druid/pull/14004#issuecomment-1586644312

   @abhishekagarwal87 
   
   On your suggestion about encouraging an action be included.  You gave a great example that explains what this PR is all about.  Specifically, think of this error message
   
   ```
   org.apache.druid.java.util.common.ISE: No default server found!
   ```
   
   If someone said that it was "Persona.USER", I would hope that the reviewer would be like, "uhhh, why will a user know what a default server is?  Your message is not well aligned with your persona" and ask for the message to be updated.
   
   If I were to adjust that exception to this new model and not change the message, I would likely make it `Persona.DEVELOPER` because it's so obtuse, only a developer would be able to actually make sense of what it means.  Of course, the exception is actually happening at a point where there is some behavior that is meaningful for the end user.
   
   We could adjust the message to be an ADMIN persona message and say
   
   ```
   Router unable to find a broker, check that brokers are up and active
   ```
   
   Or a USER persona message and say
   
   ```
   Cannot find a queryable server, contact your cluster administrator to validate that all services are operational
   ```
   
   Both of these include an action, like you suggest, but that action becomes apparent and clear once we actually identify the persona.  My *hope* is that by identifying the persona, reviewers will start to validate that the exception messages align with the intended persona and we all police each other for creating good error messages.  I.e. I'm hoping that just identifying the persona is enough to help us do better.
   
   Alternatively, for USER-facing messages, I could add the requirement for yet-another field which is "potential actions" (or something similar) which should be populated with a message describing actions that can be taken.  I'd personally like to keep it to just a message and code-review based policing.  But I'm also willing to add the field if I'm in the minority...
   
   Tagging other people whose name also appears in the conversation part of this PR to see if there are opinions one way or the other: @gianm @kfaraz @clintropolis 


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


[GitHub] [druid] cheddar commented on pull request #14004: Errors take 3

Posted by "cheddar (via GitHub)" <gi...@apache.org>.
cheddar commented on PR #14004:
URL: https://github.com/apache/druid/pull/14004#issuecomment-1596719065

   The last run of the tests was near complete (there processing tests for jdk11 and jdk17 still running, everything else, including ITs had passed), so I went ahead and merged instead of risk another set of conflicts cropping up and causing yet another CI run.


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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1233484239


##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java:
##########
@@ -1079,7 +1074,7 @@ public void testInsertLimitWithPeriodGranularityThrowsException()
                              + "LIMIT 50 "
                              + "PARTITIONED BY MONTH")
                      .setExpectedValidationErrorMatcher(CoreMatchers.allOf(
-                         CoreMatchers.instanceOf(SqlPlanningException.class),
+                         CoreMatchers.instanceOf(DruidException.class),
                          ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
                              "INSERT and REPLACE queries cannot have a LIMIT unless PARTITIONED BY is \"ALL\""))

Review Comment:
   Yes, it could.  I'm not sure which, but you are either seeing stuff left behind when I first started converting tests and hadn't yet build the Matcher to make it simpler (and I missed it in going back to simplify). OR, with fixing the conflicts after 2 months dormant, I will admit that I didn't have all of the context still in my head and perhaps did something a bit rote.  Either way, it's much simpler to switch to the `invalidSqlContains()`.



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


[GitHub] [druid] cheddar merged pull request #14004: Errors take 3

Posted by "cheddar (via GitHub)" <gi...@apache.org>.
cheddar merged PR #14004:
URL: https://github.com/apache/druid/pull/14004


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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1226105093


##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.
+ * <p>
+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception
+ * will make its way back to the user.  DruidException is always the answer to "how do I generate an error message and
+ * deliver it to the user"?
+ * <p>
+ * Every error consists of:
+ * <ul>
+ * <li>A target persona</li>
+ * <li>A categorization of the error</li>
+ * <li>An error code</li>
+ * <li>An error message</li>
+ * <li>A context (possibly empty)</li>
+ * </ul>
+ * <p>
+ * <p>
+ * The target persona indicates who the message is written for.  This is important for 2 reasons
+ * <ol>
+ *   <li>It identifies why the developer is creating the exception and who they believe can take action on it.
+ *   This context allows for code reviewers and other developers to evaluate the message with this context in mind</li>
+ *   <li>It can be used as a way to control which error messages should be routed where.  For example, a user-targetted
+ *   error message should be able to be exposed directly to the user, while an operator-targetted error message should
+ *   perhaps be routed to the operators of the system instead of the end user firing a query.</li>
+ * </ol>
+ * <p>
+ * The category indicates what kind of failure occurred.  This is leveraged to align response codes (e.g. HTTP response
+ * codes) for similar exception messages.
+ * <p>
+ * The error code is a code that indicates a grouping of error messages.  There is no forced structure around whether
+ * a specific error code can be reused for different problems or not.  That is, an error code like "adhoc" will get
+ * reused in many different places as it's the basic error code used whenever a DruidException is created in-line.  But,
+ * we might decide that a specific type of error should be identified explicitly by its error code and should only mean
+ * one thing, in which case that error code might only exist on a single error.
+ * <p>
+ * The error message is a message written targetting the target persona.  It should have values interpolated into it
+ * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information.
+ * <p>
+ * The context is a place to add extra information about the error that is not necessarily interpolated into the
+ * error message.  It's a way to carry extra information that might be useful to a developer, but not necessarily to
+ * the target persona.
+ *
+ * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}.
+ * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using
+ * forPersona, it will always be an "adhoc" error.
+ * <p>
+ * Additionally, DruidException is not intended to be directly serialized.  The intention is that something converts
+ * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that
+ * ErrorResponse is used for serialization.  DruidException carries a {@link #toErrorResponse()} method because there
+ * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that
+ * introduced DruidException.
+ */
+@NotThreadSafe
+public class DruidException extends RuntimeException
+{
+  /**
+   * Starts building an "adhoc" DruidException targetting the specific persona.
+   *
+   * @param persona the target persona of the exception message
+   * @return a builder that can be used to complete the creation of the DruidException
+   */
+  public static DruidExceptionBuilder forPersona(Persona persona)
+  {
+    return new DruidExceptionBuilder("adhoc").forPersona(persona);
+  }
+
+  /**
+   * Builds a DruidException using the provided Failure class.  The errorCode is determined by the
+   * specific Failure class being used and the Failure class is responsible for setting all other
+   * required fields of the DruidException
+   *
+   * @param failure failure implementation to use to build the DruidException
+   * @return DruidException instance built from the Failure instance provided
+   */
+  public static DruidException fromFailure(Failure failure)
+  {
+    return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode()));
+  }
+
+  private final Persona targetPersona;
+  private final Category category;
+  private final String errorCode;
+  protected final Map<String, String> context = new LinkedHashMap<>();
+
+  private DruidException(
+      Throwable cause,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      final String message
+  )
+  {
+    this(cause, errorCode, targetPersona, category, message, false);
+  }
+
+  private DruidException(
+      Throwable throwable,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      String message,
+      boolean deserialized
+  )
+  {
+    super(message, throwable, true, !deserialized);
+    this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
+    this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona");
+    this.category = Preconditions.checkNotNull(category, "category");
+  }
+
+  public DruidException withContext(String key, Object value)
+  {
+    context.put(key, value == null ? null : value.toString());
+    return this;
+  }
+
+  public DruidException withContext(Map<String, String> values)
+  {
+    this.context.putAll(values);
+    return this;
+  }
+
+  public Persona getTargetPersona()
+  {
+    return targetPersona;
+  }
+
+  public Category getCategory()
+  {
+    return category;
+  }
+
+  public String getErrorCode()
+  {
+    return errorCode;
+  }
+
+  public String getContextValue(String key)
+  {
+    return context.get(key);
+  }
+
+  public Map<String, String> getContext()
+  {
+    return context;
+  }
+
+  public int getStatusCode()
+  {
+    return category.getExpectedStatus();
+  }
+
+  /**
+   * Returns this DruidException as an ErrorResponse.  This method exists for compatibility with some older code
+   * paths that serialize out Exceptions directly using Jackson.  Instead of serializing a DruidException
+   * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be
+   * used to push across the wire.
+   * <p>
+   * As such, this method should be deleted in some future world.  Anyone wondering how to serialize and deserialize
+   * a DruidException should look at {@link ErrorResponse} and leverage that instead of this.
+   *
+   * @return an ErrorResponse
+   */
+  @JsonValue
+  public ErrorResponse toErrorResponse()
+  {
+    return new ErrorResponse(this);
+  }
+
+  /**
+   * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter
+   * with the message already on the DruidException.
+   *
+   * @param msg  Message to be prepended, can be a Java format string
+   * @param args Arguments to be passed to the message if it is a Java format string
+   * @return a new DruidException with prepended-message
+   */
+  public DruidException prependAndBuild(String msg, Object... args)
+  {
+    return new DruidException(
+        this,
+        errorCode,
+        targetPersona,
+        category,
+        StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage())
+    ).withContext(context);
+  }
+
+  /**
+   * The persona that the message on a DruidException is targetting
+   */
+  public enum Persona
+  {
+    /**
+     * Represents the end-user, a persona who is issuing queries to the Druid Query APIs
+     */
+    USER,
+    /**
+     * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query
+     * concepts without necessarily owning the infrastructure and operations of the cluster
+     */
+    ADMIN,
+    /**
+     * Represents a persona who actively owns and operates the cluster.  This persona is not assumed to understand
+     * Druid query concepts, but instead understand cluster operational concepts.
+     */
+    OPERATOR,
+    /**
+     * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase.
+     * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not
+     * possible to make a message that will make sense to a different persona.  Generally speaking, there is a hope
+     * that only DEFENSIVE error messages will target this persona.
+     */
+    DEVELOPER

Review Comment:
   It might be "file a bug" or it might just be "this error message is completely impenetrable, we know that, but don't have the time to make it so that we can make a useful error message, so we just mark it as intended for the DEVELOPER audience and move on".  It would be great if there are no errors for this persona (or if they are basically all just DEFENSIVE exceptions)...



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


[GitHub] [druid] cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "cheddar (via GitHub)" <gi...@apache.org>.
cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1231881910


##########
processing/src/test/java/org/apache/druid/error/ErrorResponseTest.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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 com.google.common.collect.ImmutableMap;
+import org.apache.druid.matchers.DruidMatchers;
+import org.apache.druid.query.QueryTimeoutException;
+import org.hamcrest.Matcher;
+import org.hamcrest.MatcherAssert;
+import org.hamcrest.Matchers;
+import org.junit.Test;
+
+import java.util.Map;
+
+public class ErrorResponseTest

Review Comment:
   yeah, they are covered primarily in the server tests.  This test also covers it some (by validating asMap, which is what eventually gets serialized), but this test was created primarily just to get coverage inside of the processing package, it doesn't actually validate much more than what the other tests already do from the server package.



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


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #14004: Errors take 3

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1231791313


##########
processing/src/main/java/org/apache/druid/common/exception/DruidException.java:
##########
@@ -22,6 +22,7 @@
 /**
  * A generic exception thrown by Druid.
  */
+@Deprecated

Review Comment:
   can you also add a note that it's deprecated in favor of the other DruidException? 



##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java:
##########
@@ -429,24 +435,30 @@ public void testNullInputs()
   @Test
   public void testArrayOfDoublesSketchIntersectOnScalarExpression()
   {
-    assertQueryIsUnplannable("SELECT DS_TUPLE_DOUBLES_INTERSECT(NULL, NULL) FROM foo",
-        "Possible error: DS_TUPLE_DOUBLES_INTERSECT can only be used on aggregates. " +
-            "It cannot be used directly on a column or on a scalar expression.");
+    assertQueryIsUnplannable(
+        "SELECT DS_TUPLE_DOUBLES_INTERSECT(NULL, NULL) FROM foo",
+        "DS_TUPLE_DOUBLES_INTERSECT can only be used on aggregates. " +

Review Comment:
   btw we had "Possible error" as a prefix in query planning errors because they are in a way best-effort guesses. During planning phase, we record instances of the planner trying to do something unsupported. and if there is nothing else to show, we just return that instance as an error. But its possible that planner didn't really need to go down that path or it tried to same very fancy planning and we end up surfacing that error instead. 



##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/sql/ArrayOfDoublesSketchSqlAggregatorTest.java:
##########
@@ -429,24 +435,30 @@ public void testNullInputs()
   @Test
   public void testArrayOfDoublesSketchIntersectOnScalarExpression()
   {
-    assertQueryIsUnplannable("SELECT DS_TUPLE_DOUBLES_INTERSECT(NULL, NULL) FROM foo",
-        "Possible error: DS_TUPLE_DOUBLES_INTERSECT can only be used on aggregates. " +
-            "It cannot be used directly on a column or on a scalar expression.");
+    assertQueryIsUnplannable(
+        "SELECT DS_TUPLE_DOUBLES_INTERSECT(NULL, NULL) FROM foo",
+        "DS_TUPLE_DOUBLES_INTERSECT can only be used on aggregates. " +

Review Comment:
   ok. I think that this got replaced with "Query planning failed for unknown reason, our best guess is this"



##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java:
##########
@@ -958,14 +955,14 @@ public void testInsertWrongTypeTimestamp()
   @Test
   public void testIncorrectInsertQuery()
   {
-    testIngestQuery().setSql(
-                         "insert into foo1 select  __time, dim1 , count(*) as cnt from foo  where dim1 is not null group by 1, 2 clustered by dim1")
-                     .setExpectedValidationErrorMatcher(CoreMatchers.allOf(
-                         CoreMatchers.instanceOf(SqlPlanningException.class),
-                         ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith(
-                             "CLUSTERED BY found before PARTITIONED BY. In Druid, the CLUSTERED BY clause must follow the PARTITIONED BY clause"))
-                     ))
-                     .verifyPlanningErrors();
+    testIngestQuery()
+        .setSql(
+            "insert into foo1 select  __time, dim1 , count(*) as cnt from foo  where dim1 is not null group by 1, 2 clustered by dim1"
+        )
+        .setExpectedValidationErrorMatcher(invalidSqlContains(
+            "LUSTERED BY found before PARTITIONED BY, CLUSTERED BY must come after the PARTITIONED BY clause"

Review Comment:
   ```suggestion
               "CLUSTERED BY found before PARTITIONED BY, CLUSTERED BY must come after the PARTITIONED BY clause"
   ```



##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java:
##########
@@ -1079,7 +1074,7 @@ public void testInsertLimitWithPeriodGranularityThrowsException()
                              + "LIMIT 50 "
                              + "PARTITIONED BY MONTH")
                      .setExpectedValidationErrorMatcher(CoreMatchers.allOf(
-                         CoreMatchers.instanceOf(SqlPlanningException.class),
+                         CoreMatchers.instanceOf(DruidException.class),
                          ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
                              "INSERT and REPLACE queries cannot have a LIMIT unless PARTITIONED BY is \"ALL\""))

Review Comment:
   could these be simplified into invalidSqlContains as well? It's find if that refactoring was a bit too much and you gave up on it after a few :P 



##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java:
##########
@@ -1673,7 +1659,7 @@ public void testGroupByWithComplexColumnThrowsUnsupportedException()
         .setSql("select unique_dim1 from foo2 group by unique_dim1")
         .setQueryContext(context)
         .setExpectedExecutionErrorMatcher(CoreMatchers.allOf(
-            CoreMatchers.instanceOf(UnsupportedSQLQueryException.class),
+            CoreMatchers.instanceOf(DruidException.class),

Review Comment:
   As we use one exception class to rule them all, were there places where there was exception-class-specific handling? I am wondering how you dealt with those. Or maybe I will find out myself as I move below. 



##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java:
##########
@@ -1641,8 +1622,13 @@ public void testTimeColumnAggregationFromExtern() throws IOException
                 + "FROM kttm_data "
                 + "GROUP BY 1")
         .setExpectedValidationErrorMatcher(
-            ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
-                "LATEST() aggregator depends on __time column"))
+            new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "adhoc")
+                .expectMessageIs(
+                    "Query planning failed for unknown reason, our best guess is this "

Review Comment:
   How about "Query planning failed for unknown reason, most likely the reason is this"



##########
processing/src/main/java/org/apache/druid/error/QueryExceptionCompat.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.query.QueryException;
+
+public class QueryExceptionCompat extends DruidException.Failure

Review Comment:
   this class could use some javadocs. 



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


[GitHub] [druid] clintropolis commented on a diff in pull request #14004: Errors take 3

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1163659261


##########
processing/src/test/java/org/apache/druid/matchers/DMatchers.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.matchers;
+
+import org.hamcrest.Matcher;
+
+import java.util.function.Function;
+
+public class DMatchers

Review Comment:
   nit: what is `DMatchers`? (javadoc or better name needed?)



##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java:
##########
@@ -123,7 +129,7 @@ private SqlNode createArrayLiteral(Object value)
     List<SqlNode> args = new ArrayList<>(list.size());
     for (Object element : list) {
       if (element == null) {
-        throw new IAE("An array parameter cannot contain null values");
+        throw InvalidSqlInput.exception("An array parameter [%s] cannot contain null values", posn + 1);

Review Comment:
   hmm, this message seems a bit confusing, should it be ordered in some manner so it reads like "parameter in position [n] is an array but has null values which are illegal for some reason" (side note: why aren't they allowed i wonder?)



##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java:
##########
@@ -125,12 +126,12 @@ public void reduce(
               // the query can execute.
               double exprResultDouble = exprResult.asDouble();
               if (Double.isNaN(exprResultDouble) || Double.isInfinite(exprResultDouble)) {
-                String expression = druidExpression.getExpression();
-                throw new UnsupportedSQLQueryException("'%s' evaluates to '%s' that is not supported in SQL. You can either cast the expression as BIGINT ('CAST(%s as BIGINT)') or VARCHAR ('CAST(%s as VARCHAR)') or change the expression itself",
-                    expression,
-                    Double.toString(exprResultDouble),
-                    expression,
-                    expression);
+                throw InvalidSqlInput.exception(
+                        "Expression [%s] evaluates to an unsupported value [%s], expected something that"
+                        + " can be a Double.  Consider casting with 'CAST(<col> AS BIGINT)'",
+                        druidExpression.getExpression(),
+                        exprResultDouble
+                    );

Review Comment:
   hmm, i know this isn't new, but these messages barely feel like user errors since it happens when calcite is trying to reduce some constant expression and we blow it for one reason or another. Looking back this seems to originally come from #11409 for the one doing numbers and then copied to arrays in #11968



##########
processing/src/main/java/org/apache/druid/error/InvalidSqlInput.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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;
+
+public class InvalidSqlInput extends InvalidInput

Review Comment:
   is it weird this is defined in processing but sql is not here?



##########
sql/src/main/java/org/apache/druid/sql/calcite/planner/SqlParameterizerShuttle.java:
##########
@@ -105,14 +106,19 @@ public SqlNode visit(SqlDynamicParam param)
     }
   }
 
+  private static DruidException unbound(SqlDynamicParam param)
+  {
+    return InvalidSqlInput.exception("No value bound for parameter (position [%s])", param.getIndex() + 1);
+  }
+

Review Comment:
   nit: should this and the rel parameterizer shuttle share?



##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.
+ * <p>
+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception
+ * will make its way back to the user.  DruidException is always the answer to "how do I generate an error message and
+ * deliver it to the user"?
+ * <p>
+ * Every error consists of:
+ * <ul>
+ * <li>A target persona</li>
+ * <li>A categorization of the error</li>
+ * <li>An error code</li>
+ * <li>An error message</li>
+ * <li>A context (possibly empty)</li>
+ * </ul>
+ * <p>
+ * <p>
+ * The target persona indicates who the message is written for.  This is important for 2 reasons
+ * <ol>
+ *   <li>It identifies why the developer is creating the exception and who they believe can take action on it.
+ *   This context allows for code reviewers and other developers to evaluate the message with this context in mind</li>
+ *   <li>It can be used as a way to control which error messages should be routed where.  For example, a user-targetted
+ *   error message should be able to be exposed directly to the user, while an operator-targetted error message should
+ *   perhaps be routed to the operators of the system instead of the end user firing a query.</li>
+ * </ol>
+ * <p>
+ * The category indicates what kind of failure occurred.  This is leveraged to align response codes (e.g. HTTP response
+ * codes) for similar exception messages.
+ * <p>
+ * The error code is a code that indicates a grouping of error messages.  There is no forced structure around whether
+ * a specific error code can be reused for different problems or not.  That is, an error code like "adhoc" will get
+ * reused in many different places as it's the basic error code used whenever a DruidException is created in-line.  But,
+ * we might decide that a specific type of error should be identified explicitly by its error code and should only mean
+ * one thing, in which case that error code might only exist on a single error.
+ * <p>
+ * The error message is a message written targetting the target persona.  It should have values interpolated into it
+ * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information.
+ * <p>
+ * The context is a place to add extra information about the error that is not necessarily interpolated into the
+ * error message.  It's a way to carry extra information that might be useful to a developer, but not necessarily to
+ * the target persona.
+ *
+ * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}.
+ * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using
+ * forPersona, it will always be an "adhoc" error.
+ * <p>
+ * Additionally, DruidException is not intended to be directly serialized.  The intention is that something converts
+ * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that
+ * ErrorResponse is used for serialization.  DruidException carries a {@link #toErrorResponse()} method because there
+ * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that
+ * introduced DruidException.
+ */
+@NotThreadSafe
+public class DruidException extends RuntimeException
+{
+  /**
+   * Starts building an "adhoc" DruidException targetting the specific persona.
+   *
+   * @param persona the target persona of the exception message
+   * @return a builder that can be used to complete the creation of the DruidException
+   */
+  public static DruidExceptionBuilder forPersona(Persona persona)
+  {
+    return new DruidExceptionBuilder("adhoc").forPersona(persona);
+  }
+
+  /**
+   * Builds a DruidException using the provided Failure class.  The errorCode is determined by the
+   * specific Failure class being used and the Failure class is responsible for setting all other
+   * required fields of the DruidException
+   *
+   * @param failure failure implementation to use to build the DruidException
+   * @return DruidException instance built from the Failure instance provided
+   */
+  public static DruidException fromFailure(Failure failure)
+  {
+    return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode()));
+  }
+
+  private final Persona targetPersona;
+  private final Category category;
+  private final String errorCode;
+  protected final Map<String, String> context = new LinkedHashMap<>();
+
+  private DruidException(
+      Throwable cause,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      final String message
+  )
+  {
+    this(cause, errorCode, targetPersona, category, message, false);
+  }
+
+  private DruidException(
+      Throwable throwable,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      String message,
+      boolean deserialized
+  )
+  {
+    super(message, throwable, true, !deserialized);
+    this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
+    this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona");
+    this.category = Preconditions.checkNotNull(category, "category");
+  }
+
+  public DruidException withContext(String key, Object value)
+  {
+    context.put(key, value == null ? null : value.toString());
+    return this;
+  }
+
+  public DruidException withContext(Map<String, String> values)
+  {
+    this.context.putAll(values);
+    return this;
+  }
+
+  public Persona getTargetPersona()
+  {
+    return targetPersona;
+  }
+
+  public Category getCategory()
+  {
+    return category;
+  }
+
+  public String getErrorCode()
+  {
+    return errorCode;
+  }
+
+  public String getContextValue(String key)
+  {
+    return context.get(key);
+  }
+
+  public Map<String, String> getContext()
+  {
+    return context;
+  }
+
+  public int getStatusCode()
+  {
+    return category.getExpectedStatus();
+  }
+
+  /**
+   * Returns this DruidException as an ErrorResponse.  This method exists for compatibility with some older code
+   * paths that serialize out Exceptions directly using Jackson.  Instead of serializing a DruidException
+   * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be
+   * used to push across the wire.
+   * <p>
+   * As such, this method should be deleted in some future world.  Anyone wondering how to serialize and deserialize
+   * a DruidException should look at {@link ErrorResponse} and leverage that instead of this.
+   *
+   * @return an ErrorResponse
+   */
+  @JsonValue
+  public ErrorResponse toErrorResponse()
+  {
+    return new ErrorResponse(this);
+  }
+
+  /**
+   * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter
+   * with the message already on the DruidException.
+   *
+   * @param msg  Message to be prepended, can be a Java format string
+   * @param args Arguments to be passed to the message if it is a Java format string
+   * @return a new DruidException with prepended-message
+   */
+  public DruidException prependAndBuild(String msg, Object... args)
+  {
+    return new DruidException(
+        this,
+        errorCode,
+        targetPersona,
+        category,
+        StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage())
+    ).withContext(context);
+  }
+
+  /**
+   * The persona that the message on a DruidException is targetting
+   */
+  public enum Persona
+  {
+    /**
+     * Represents the end-user, a persona who is issuing queries to the Druid Query APIs
+     */
+    USER,
+    /**
+     * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query
+     * concepts without necessarily owning the infrastructure and operations of the cluster
+     */
+    ADMIN,
+    /**
+     * Represents a persona who actively owns and operates the cluster.  This persona is not assumed to understand
+     * Druid query concepts, but instead understand cluster operational concepts.
+     */
+    OPERATOR,
+    /**
+     * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase.
+     * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not
+     * possible to make a message that will make sense to a different persona.  Generally speaking, there is a hope
+     * that only DEFENSIVE error messages will target this persona.
+     */
+    DEVELOPER

Review Comment:
   is this effectively "file a bug report"?



##########
sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidLogicalValuesRule.java:
##########
@@ -126,14 +126,18 @@ static Object getValueFromLiteral(RexLiteral literal, PlannerContext plannerCont
         return Calcites.calciteDateTimeLiteralToJoda(literal, plannerContext.getTimeZone()).getMillis();
       case NULL:
         if (!literal.isNull()) {
-          throw new UnsupportedSQLQueryException("Query has a non-null constant but is of NULL type.");
+          throw InvalidSqlInput.exception("Expected a NULL literal, but got non-null constant [%s]", literal);

Review Comment:
   i know this isn't new... but can this.. happen? seems strange



##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.

Review Comment:
   Nice description but I feel like it doesn't quite answer the questions I have as a developer who might be wanting to use this (apologies if this was covered in previous discussions/iterations of this PR...)
   
   in what types of places should I as a developer throws these errors instead of like `ISE` or `IAE`? Should these only be used at surface layers, or is it fine for like a historical to throw one of these too? If I do throw it on a historical, should a broker do stuff to it (like merge errors from multiple hosts perhaps or decorate any additional context?) Are these intended to be used for both synchronous http responses and also async error status values that might be user facing? Just for queries or also other APIs and maybe someday ingestion tasks too?
   
   I think clearing up some of these things will help make this actually get traction (this was also imo one of the harder parts about `QueryException` as well, like .. when should i actually throw one instead of throwing something else)



##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.
+ * <p>
+ * Said another way, when a developer builds a DruidException in the code, they should be confident that the exception
+ * will make its way back to the user.  DruidException is always the answer to "how do I generate an error message and
+ * deliver it to the user"?
+ * <p>
+ * Every error consists of:
+ * <ul>
+ * <li>A target persona</li>
+ * <li>A categorization of the error</li>
+ * <li>An error code</li>
+ * <li>An error message</li>
+ * <li>A context (possibly empty)</li>
+ * </ul>
+ * <p>
+ * <p>
+ * The target persona indicates who the message is written for.  This is important for 2 reasons
+ * <ol>
+ *   <li>It identifies why the developer is creating the exception and who they believe can take action on it.
+ *   This context allows for code reviewers and other developers to evaluate the message with this context in mind</li>
+ *   <li>It can be used as a way to control which error messages should be routed where.  For example, a user-targetted
+ *   error message should be able to be exposed directly to the user, while an operator-targetted error message should
+ *   perhaps be routed to the operators of the system instead of the end user firing a query.</li>
+ * </ol>
+ * <p>
+ * The category indicates what kind of failure occurred.  This is leveraged to align response codes (e.g. HTTP response
+ * codes) for similar exception messages.
+ * <p>
+ * The error code is a code that indicates a grouping of error messages.  There is no forced structure around whether
+ * a specific error code can be reused for different problems or not.  That is, an error code like "adhoc" will get
+ * reused in many different places as it's the basic error code used whenever a DruidException is created in-line.  But,
+ * we might decide that a specific type of error should be identified explicitly by its error code and should only mean
+ * one thing, in which case that error code might only exist on a single error.
+ * <p>
+ * The error message is a message written targetting the target persona.  It should have values interpolated into it
+ * in order to be as meaningful as possible for the target persona without leaking potentially sensitive information.
+ * <p>
+ * The context is a place to add extra information about the error that is not necessarily interpolated into the
+ * error message.  It's a way to carry extra information that might be useful to a developer, but not necessarily to
+ * the target persona.
+ *
+ * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * A DruidException can be built from one of 2 static methods: {@link #forPersona} or {@link #fromFailure(Failure)}.
+ * The only way to set a specific error code is to build a DruidException from a Failure, when built in-line using
+ * forPersona, it will always be an "adhoc" error.
+ * <p>
+ * Additionally, DruidException is not intended to be directly serialized.  The intention is that something converts
+ * it into an {@link ErrorResponse} first using {@link ErrorResponse#ErrorResponse(DruidException)} and then that
+ * ErrorResponse is used for serialization.  DruidException carries a {@link #toErrorResponse()} method because there
+ * are some code paths that directly serialize Exceptions and adjusting them was deemed out-of-scope for the PR that
+ * introduced DruidException.
+ */
+@NotThreadSafe
+public class DruidException extends RuntimeException
+{
+  /**
+   * Starts building an "adhoc" DruidException targetting the specific persona.
+   *
+   * @param persona the target persona of the exception message
+   * @return a builder that can be used to complete the creation of the DruidException
+   */
+  public static DruidExceptionBuilder forPersona(Persona persona)
+  {
+    return new DruidExceptionBuilder("adhoc").forPersona(persona);
+  }
+
+  /**
+   * Builds a DruidException using the provided Failure class.  The errorCode is determined by the
+   * specific Failure class being used and the Failure class is responsible for setting all other
+   * required fields of the DruidException
+   *
+   * @param failure failure implementation to use to build the DruidException
+   * @return DruidException instance built from the Failure instance provided
+   */
+  public static DruidException fromFailure(Failure failure)
+  {
+    return failure.makeException(new DruidExceptionBuilder(failure.getErrorCode()));
+  }
+
+  private final Persona targetPersona;
+  private final Category category;
+  private final String errorCode;
+  protected final Map<String, String> context = new LinkedHashMap<>();
+
+  private DruidException(
+      Throwable cause,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      final String message
+  )
+  {
+    this(cause, errorCode, targetPersona, category, message, false);
+  }
+
+  private DruidException(
+      Throwable throwable,
+      final String errorCode,
+      Persona targetPersona,
+      Category category,
+      String message,
+      boolean deserialized
+  )
+  {
+    super(message, throwable, true, !deserialized);
+    this.errorCode = Preconditions.checkNotNull(errorCode, "errorCode");
+    this.targetPersona = Preconditions.checkNotNull(targetPersona, "targetPersona");
+    this.category = Preconditions.checkNotNull(category, "category");
+  }
+
+  public DruidException withContext(String key, Object value)
+  {
+    context.put(key, value == null ? null : value.toString());
+    return this;
+  }
+
+  public DruidException withContext(Map<String, String> values)
+  {
+    this.context.putAll(values);
+    return this;
+  }
+
+  public Persona getTargetPersona()
+  {
+    return targetPersona;
+  }
+
+  public Category getCategory()
+  {
+    return category;
+  }
+
+  public String getErrorCode()
+  {
+    return errorCode;
+  }
+
+  public String getContextValue(String key)
+  {
+    return context.get(key);
+  }
+
+  public Map<String, String> getContext()
+  {
+    return context;
+  }
+
+  public int getStatusCode()
+  {
+    return category.getExpectedStatus();
+  }
+
+  /**
+   * Returns this DruidException as an ErrorResponse.  This method exists for compatibility with some older code
+   * paths that serialize out Exceptions directly using Jackson.  Instead of serializing a DruidException
+   * directly, code should be structured to take the DruidException and build an ErrorResponse from it to be
+   * used to push across the wire.
+   * <p>
+   * As such, this method should be deleted in some future world.  Anyone wondering how to serialize and deserialize
+   * a DruidException should look at {@link ErrorResponse} and leverage that instead of this.
+   *
+   * @return an ErrorResponse
+   */
+  @JsonValue
+  public ErrorResponse toErrorResponse()
+  {
+    return new ErrorResponse(this);
+  }
+
+  /**
+   * Builds a new DruidException with a message that is the result of prepending the message passed as a parameter
+   * with the message already on the DruidException.
+   *
+   * @param msg  Message to be prepended, can be a Java format string
+   * @param args Arguments to be passed to the message if it is a Java format string
+   * @return a new DruidException with prepended-message
+   */
+  public DruidException prependAndBuild(String msg, Object... args)
+  {
+    return new DruidException(
+        this,
+        errorCode,
+        targetPersona,
+        category,
+        StringUtils.format("%s: %s", StringUtils.nonStrictFormat(msg, args), getMessage())
+    ).withContext(context);
+  }
+
+  /**
+   * The persona that the message on a DruidException is targetting
+   */
+  public enum Persona
+  {
+    /**
+     * Represents the end-user, a persona who is issuing queries to the Druid Query APIs
+     */
+    USER,
+    /**
+     * Represents an administrative user, a persona who is interacting with admin APIs and understands Druid query
+     * concepts without necessarily owning the infrastructure and operations of the cluster
+     */
+    ADMIN,
+    /**
+     * Represents a persona who actively owns and operates the cluster.  This persona is not assumed to understand
+     * Druid query concepts, but instead understand cluster operational concepts.
+     */
+    OPERATOR,
+    /**
+     * Represents someone who has all of the context and knowledge to be actively diving into the Druid codebase.
+     * This persona exists as a catch-all for anything that is so deep and technically in the weeds that it is not
+     * possible to make a message that will make sense to a different persona.  Generally speaking, there is a hope
+     * that only DEFENSIVE error messages will target this persona.
+     */
+    DEVELOPER
+  }
+
+  /**
+   * Category of error.  The simplest way to describe this is that it exists as a classification of errors that
+   * enables us to identify the expected response code (e.g. HTTP status code) of a specific DruidException
+   */
+  public enum Category
+  {
+    /**
+     * Means that the exception is being created defensively, because we want to validate something but expect that
+     * it should never actually be hit.  Using this category is good to provide an indication to future reviewers and
+     * developers that the case being checked is not intended to actually be able to occur in the wild.
+     */
+    DEFENSIVE(500),
+    /**
+     * Means that the input provided was malformed in some way.  Generally speaking, it is hoped that errors of this
+     * category have messages written either targetting the USER or ADMIN personas as those are the general users
+     * of the APIs who could generate invalid inputs.
+     */
+    INVALID_INPUT(400),
+    /**
+     * Means that the error is a problem with authorization.
+     */
+    UNAUTHORIZED(401),
+    /**
+     * Means that some capacity limit was exceeded, this could be due to throttling or due to some system limit
+     */
+    CAPACITY_EXCEEDED(429),
+    /**
+     * Means that the query was canceled for some reason
+     */
+    CANCELED(500),
+    /**
+     * Indicates a server-side failure of some sort at runtime
+     */
+    RUNTIME_FAILURE(500),
+    /**
+     * A timeout happened
+     */
+    TIMEOUT(504),
+    /**
+     * Indicates some unsupported behavior was requested.  TODO
+     */
+    UNSUPPORTED(501),
+    /**
+     * A catch-all for any time when we cannot come up with a meaningful categorization.  This is hopefully only
+     * used when converting generic exceptions from frameworks and libraries that we do not control into DruidExcpetions
+     */
+    UNCATEGORIZED(500);
+
+    private final int expectedStatus;
+
+    Category(int expectedStatus)
+    {
+      this.expectedStatus = expectedStatus;
+    }
+
+    public int getExpectedStatus()
+    {
+      return expectedStatus;
+    }
+  }
+
+  public static class DruidExceptionBuilder
+  {
+    private String errorCode;
+    private Persona targetPersona;
+    private Category category;
+
+    private boolean deserialized = false;
+
+    private DruidExceptionBuilder(String errorCode)
+    {
+      this.errorCode = errorCode;
+    }
+
+    public DruidExceptionBuilder forPersona(Persona targetPersona)
+    {
+      this.targetPersona = targetPersona;
+      return this;
+    }
+
+    public DruidExceptionBuilder ofCategory(Category category)
+    {
+      this.category = category;
+      return this;
+    }
+
+    /**
+     * Exists for ErrorMessage to be able to indicate that the exception was deserialized and (therefore)
+     * should not carry any stack-trace as the stack-trace generated would be to the deserialization code rather than
+     * the actual error.
+     *
+     * @return the builder
+     */
+    DruidExceptionBuilder wasDeserialized()
+    {
+      this.deserialized = true;
+      return this;
+    }
+
+    public DruidException build(String formatMe, Object... vals)
+    {
+      return build(null, formatMe, vals);
+    }
+
+    public DruidException build(Throwable cause, String formatMe, Object... vals)
+    {
+      return new DruidException(
+          cause,
+          errorCode,
+          targetPersona,
+          category,
+          StringUtils.nonStrictFormat(formatMe, vals),
+          deserialized
+      );
+    }
+  }
+
+  public abstract static class Failure

Review Comment:
   nit: javadocs



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


[GitHub] [druid] cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "cheddar (via GitHub)" <gi...@apache.org>.
cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1233473366


##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java:
##########
@@ -1641,8 +1622,13 @@ public void testTimeColumnAggregationFromExtern() throws IOException
                 + "FROM kttm_data "
                 + "GROUP BY 1")
         .setExpectedValidationErrorMatcher(
-            ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString(
-                "LATEST() aggregator depends on __time column"))
+            new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "adhoc")
+                .expectMessageIs(
+                    "Query planning failed for unknown reason, our best guess is this "

Review Comment:
   I'm anti that messaging because it pretends like the following error message has a high likelihood of being the cause.  My experience has told me that this is rarely the case and most likely this error message means "something happened, I'm gonna tell you something that is just as likely to be wrong as it is right, good luck".  So I don't want to pretend like it's giving people a good answer when it generally is not.



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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #14004: Errors take 3

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1226106032


##########
processing/src/main/java/org/apache/druid/error/DruidException.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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 com.fasterxml.jackson.annotation.JsonValue;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+
+import javax.annotation.concurrent.NotThreadSafe;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * Represents an error condition exposed to the user and/or operator of Druid.  Given that a DruidException is intended
+ * to be delivered to the end user, it should generally never be caught.  DruidExceptions are generated at terminal
+ * points where the operation that was happening cannot make forward progress.  As such, the only reason to catch a
+ * DruidException is if the code has some extra context that it wants to add to the message of the DruidException using
+ * {@link #prependAndBuild(String, Object...)}.  If code wants to catch and handle an exception instead, it should not
+ * be using the DruidException.

Review Comment:
   Generally speaking, IAE/ISE would disappear and all be replaced by `DruidException`.  We can/definitely would continue to have rules that map IAE/ISE into `DruidException` (probably as a DEVELOPER-focused persona exception), but in the fullness of time, we'd probably want IAE/ISE to be replaced with `DruidException` instances.



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


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #14004: Errors take 3

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1229591223


##########
server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java:
##########
@@ -442,27 +446,21 @@
     );
 
     for (Map.Entry<String, String> entry : invalidCharToDataSourceName.entrySet()) {
-      testInvalidWhitespaceDatasourceHelper(entry.getValue(), entry.getKey());
-    }
-  }
-
-  private void testInvalidWhitespaceDatasourceHelper(String dataSource, String invalidChar)
-  {
-    String testFailMsg = "dataSource contain invalid whitespace character: " + invalidChar;
-    try {
-      DataSchema schema = new DataSchema(
-          dataSource,
-          Collections.emptyMap(),
-          null,
-          null,
-          null,
-          jsonMapper
+      String dataSource = entry.getValue();
+      final String msg = StringUtils.format(
+          "Invalid value for field [dataSource]: Value [%s] contains illegal whitespace characters.  Only space is allowed.",
+          dataSource
+      );
+      DruidExceptionMatcher.invalidInput().expectMessageIs(msg).assertThrowsAndMatches(
+          () -> new DataSchema(
+              dataSource,
+              Collections.emptyMap(),
+              null,
+              null,
+              null,
+              jsonMapper
+          )

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [DataSchema.DataSchema](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5085)



##########
server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java:
##########
@@ -413,22 +415,24 @@
         ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
     );
 
-    expectedException.expect(CoreMatchers.instanceOf(IllegalArgumentException.class));
-    expectedException.expectMessage(
-        "dataSource cannot be null or empty. Please provide a dataSource."
-    );
-
-    DataSchema schema = new DataSchema(
-        "",
-        parser,
-        new AggregatorFactory[]{
-            new DoubleSumAggregatorFactory("metric1", "col1"),
-            new DoubleSumAggregatorFactory("metric2", "col2"),
-            },
-        new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
-        null,
-        jsonMapper
-    );
+    DruidExceptionMatcher
+        .invalidInput()
+        .expectMessageIs("Invalid value for field [dataSource]: must not be null")
+        .assertThrowsAndMatches(
+            () -> new DataSchema(
+                "",
+                parser,
+                new AggregatorFactory[]{
+                    new DoubleSumAggregatorFactory("metric1", "col1"),
+                    new DoubleSumAggregatorFactory("metric2", "col2"),
+                    },
+                new ArbitraryGranularitySpec(
+                    Granularities.DAY,
+                    ImmutableList.of(Intervals.of("2014/2015"))
+                ),
+                null,
+                jsonMapper
+            ));

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [DataSchema.DataSchema](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5084)



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


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #14004: Errors take 3

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #14004:
URL: https://github.com/apache/druid/pull/14004#discussion_r1230361239


##########
processing/src/test/java/org/apache/druid/matchers/DruidMatchers.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.matchers;
+
+import org.hamcrest.Matcher;
+import org.hamcrest.Matchers;
+
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.function.Function;
+
+public class DruidMatchers
+{
+  public static <T, S> LambdaMatcher<T, S> fn(String name, Function<T, S> fn, Matcher<S> matcher)
+  {
+    return new LambdaMatcher<>(name + ": ", fn, matcher);
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public static <K, V> Matcher<Map<? extends K, ? extends V>> mapMatcher(Object... keysAndValues)
+  {
+    ArrayList<Matcher<Map<? extends K, ? extends V>>> entryMatchers = new ArrayList<>();
+    for (int i = 0; i < keysAndValues.length; i += 2) {
+      entryMatchers.add(Matchers.hasEntry((K) keysAndValues[i], (V) keysAndValues[i + 1]));

Review Comment:
   ## Array index out of bounds
   
   This array access might be out of bounds, as the index might be equal to the array length.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/5086)



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