You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/08/18 19:14:51 UTC

[GitHub] [ignite-3] PakhomovAlexander opened a new pull request, #1024: IGNITE-17090 Show user-friendly message in sql command

PakhomovAlexander opened a new pull request, #1024:
URL: https://github.com/apache/ignite-3/pull/1024

   Previously sql command showed an abstract error message like
   `SQL query parsing erPreviously sql command showed an abstract error message like
   `SQL query parsing error` and in was hard to understand what
   was wrong. Now it shows the root cause of the issue.
   
   The format of the error message is still being different from
   time to time, and we have to think about the error message
   formatting in sql-engine.ror` and in was hard to understand what
   was wrong. Now it shows the root cause of the issue.
   
   The format of the error message is still being different from
   time to time, and we have to think about the error message
   formatting in sql-engine.
   
   https://issues.apache.org/jira/browse/IGNITE-17090


-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] zstan commented on a diff in pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
zstan commented on code in PR #1024:
URL: https://github.com/apache/ignite-3/pull/1024#discussion_r965600618


##########
modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java:
##########
@@ -272,10 +273,14 @@ public CompletableFuture<JdbcMetaPrimaryKeysResult> primaryKeysMetaAsync(JdbcMet
      * @return StringWriter filled with exception.
      */
     private StringWriter getWriterWithStackTrace(Throwable t) {
+        String message = ExceptionUtils.unwrapCause(t).getMessage();
         StringWriter sw = new StringWriter();
         PrintWriter pw = new PrintWriter(sw);
 
-        t.printStackTrace(pw);
+        pw.print(message);
+
+        // check if stacktrace is needed
+        // t.printStackTrace(pw);

Review Comment:
   remove it plz



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Pochatkin commented on a diff in pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
Pochatkin commented on code in PR #1024:
URL: https://github.com/apache/ignite-3/pull/1024#discussion_r964060262


##########
modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java:
##########
@@ -272,10 +273,17 @@ public CompletableFuture<JdbcMetaPrimaryKeysResult> primaryKeysMetaAsync(JdbcMet
      * @return StringWriter filled with exception.
      */
     private StringWriter getWriterWithStackTrace(Throwable t) {
+        String message = null;
+        if (t instanceof CompletionException) {

Review Comment:
   Done



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] sk0x50 commented on a diff in pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
sk0x50 commented on code in PR #1024:
URL: https://github.com/apache/ignite-3/pull/1024#discussion_r966746894


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java:
##########
@@ -803,7 +805,13 @@ public static SqlNodeList parse(String qry, SqlParser.Config parserCfg) {
         try {
             return parse(new SourceStringReader(qry), parserCfg);
         } catch (SqlParseException e) {
-            throw new SqlException(QUERY_INVALID_ERR, "Failed to parse query", e);
+            String message = "Failed to parse query";

Review Comment:
   I would prefer the following:
   ```
               throw withCauseAndCode(
                       SqlException::new,
                       QUERY_INVALID_ERR,
                       "Failed to parse query: " + extractCauseMessage(e.getMessage()),
                       e);
   ```



##########
modules/cli/src/integrationTest/java/org/apache/ignite/cli/commands/sql/ItSqlCommandTest.java:
##########
@@ -63,8 +63,8 @@ void wrongJdbcUrl() {
         assertAll(
                 () -> assertExitCodeIs(1),
                 this::assertOutputIsEmpty,
-                // TODO: https://issues.apache.org/jira/browse/IGNITE-17090
-                () -> assertErrOutputIs(CLIENT_CONNECTION_FAILED_MESSAGE + System.lineSeparator())
+                () -> assertErrOutputContains(CLIENT_CONNECTION_FAILED_MESSAGE),
+                () -> assertErrOutputContains("no-such-host.com: Name or service not known")

Review Comment:
   Well, this message will be "localized". For instance it can be `Этот хост неизвестен (no-such-host.com)` instead of `no-such-host.com: Name or service not known`. Perhaps, we just only need to check `() -> assertErrOutputContains(CLIENT_CONNECTION_FAILED_MESSAGE)`. It is up to you. Both variants are OK to me.



##########
modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcStatement.java:
##########
@@ -734,16 +733,6 @@ public final void timeout(int timeout) throws SQLException {
     }
 
     private static SQLException toSqlException(CompletionException e) {
-        if (e.getCause() instanceof IgniteException) {
-            IgniteException cause = (IgniteException) e.getCause();
-            String message = cause.getMessage();
-
-            if (message != null) {
-                if (message.contains("Failed to parse query")) {
-                    return new SQLException("Sql query execution failed.", SqlStateCode.PARSING_EXCEPTION, e);
-                }
-            }
-        }
-        return new SQLException("Internal server error.", SqlStateCode.INTERNAL_ERROR, e);
+        return new SQLException(e.getCause());

Review Comment:
   It does not seem correct. You should not lose original `CompletionException` because it will lead to the fact that its stack frame will be lost as well. Let's change this line to `return new SQLException(e));`



##########
modules/core/src/main/java/org/apache/ignite/lang/ErrorGroup.java:
##########
@@ -201,6 +201,13 @@ public static String errorMessage(UUID traceId, int code, String message) {
      * @return New error message with predefined prefix.
      */
     public static String errorMessage(UUID traceId, String groupName, int code, String message) {
+        if (message != null) {

Review Comment:
   Nice catch! I think we don't need to create a new string and replace special characters. The `DOTALL` option should do the trick.
   ```
       /** Error message pattern. */
       private static final Pattern EXCEPTION_MESSAGE_PATTERN =
               Pattern.compile(..., DOTALL);
   ```



##########
modules/cli/src/main/java/org/apache/ignite/cli/core/exception/handler/SqlExceptionHandler.java:
##########
@@ -38,9 +47,46 @@ public class SqlExceptionHandler implements ExceptionHandler<SQLException> {
     public static final String CLIENT_CONNECTION_FAILED_MESSAGE = "Connection failed";
 
     public static final String CONNECTION_BROKE_MESSAGE = "Connection error";
+    public static final String UNRECOGNIZED_ERROR_MESSAGE = "Unrecognized error while processing SQL query ";
+
+    private final Map<Integer, Function<IgniteException, ErrorComponentBuilder>> sqlExceptionMappers = new HashMap<>();
+
+    /** Default constructor. */
+    public SqlExceptionHandler() {
+        sqlExceptionMappers.put(Client.CONNECTION_ERR, this::connectionErrUiComponent);
+        sqlExceptionMappers.put(Sql.QUERY_INVALID_ERR, this::invalidQueryErrUiComponent);
+    }
+
+    private ErrorComponentBuilder invalidQueryErrUiComponent(IgniteException e) {
+        return fromExWithHeader(e, PARSING_ERROR_MESSAGE);
+    }
+
+    private ErrorComponentBuilder unrecognizedErrComponent(IgniteException e) {
+        return fromExWithHeader(e, UNRECOGNIZED_ERROR_MESSAGE);
+    }
+
+    private ErrorComponentBuilder connectionErrUiComponent(IgniteException e) {
+        if (e.getCause() instanceof IgniteClientConnectionException) {
+            return fromExWithHeader((IgniteClientConnectionException) e.getCause(), CLIENT_CONNECTION_FAILED_MESSAGE);
+        }
+
+        return fromExWithHeader(e, CLIENT_CONNECTION_FAILED_MESSAGE);
+    }
+
+    private static ErrorComponentBuilder fromExWithHeader(IgniteException e, String header) {
+        return ErrorUiComponent.builder()
+                .header(header)
+                .errorCode(e.codeAsString())
+                .traceId(e.traceId())
+                .details(ErrorGroup.extractCauseMessage(e.getMessage()));
+    }
 
     @Override
     public int handle(ExceptionWriter err, SQLException e) {
+        if (e.getCause() instanceof IgniteException) {

Review Comment:
   In order to properly handle `ExecutionException` and `CompletionException` I would propose:
   ```
           Throwable unwrappedCause = ExceptionUtils.unwrapCause(e.getCause());
           if (unwrappedCause instanceof IgniteException) {
               return handleIgniteException(err, (IgniteException) unwrappedCause);
           }
   ```
   
   By the way, do we need to take care of `IgniteCheckedException`'s?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] ygerzhedovich commented on a diff in pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
ygerzhedovich commented on code in PR #1024:
URL: https://github.com/apache/ignite-3/pull/1024#discussion_r965612242


##########
modules/client/src/main/java/org/apache/ignite/internal/jdbc/JdbcStatement.java:
##########
@@ -734,16 +733,6 @@ public final void timeout(int timeout) throws SQLException {
     }
 
     private static SQLException toSqlException(CompletionException e) {
-        if (e.getCause() instanceof IgniteException) {
-            IgniteException cause = (IgniteException) e.getCause();
-            String message = cause.getMessage();
-
-            if (message != null) {
-                if (message.contains("Failed to parse query")) {
-                    return new SQLException("Sql query execution failed.", SqlStateCode.PARSING_EXCEPTION, e);
-                }
-            }
-        }
-        return new SQLException("Internal server error.", SqlStateCode.INTERNAL_ERROR, e);
+        return new SQLException(e.getCause());

Review Comment:
   it looks bad for me, we lose the code



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] korlov42 commented on a diff in pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
korlov42 commented on code in PR #1024:
URL: https://github.com/apache/ignite-3/pull/1024#discussion_r963089629


##########
modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java:
##########
@@ -272,10 +273,17 @@ public CompletableFuture<JdbcMetaPrimaryKeysResult> primaryKeysMetaAsync(JdbcMet
      * @return StringWriter filled with exception.
      */
     private StringWriter getWriterWithStackTrace(Throwable t) {
+        String message = null;
+        if (t instanceof CompletionException) {

Review Comment:
   It would be better to use `org.apache.ignite.internal.util.ExceptionUtils#unwrapCause`



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on code in PR #1024:
URL: https://github.com/apache/ignite-3/pull/1024#discussion_r954259809


##########
modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java:
##########
@@ -272,10 +275,21 @@ public CompletableFuture<JdbcMetaPrimaryKeysResult> primaryKeysMetaAsync(JdbcMet
      * @return StringWriter filled with exception.
      */
     private StringWriter getWriterWithStackTrace(Throwable t) {
+        String message = null;
+        if (t instanceof CompletionException) {
+            if (t.getCause() instanceof IgniteException
+                    || t.getCause() instanceof CalciteContextException
+                    || t.getCause() instanceof QueryValidationException) {
+                message = t.getCause().getMessage();
+            }
+        }

Review Comment:
   ```suggestion
           if (t instanceof CompletionException) {
               message = t.getCause().getMessage();
           }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] asfgit closed pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #1024: IGNITE-17090 Show user-friendly message in sql command
URL: https://github.com/apache/ignite-3/pull/1024


-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] valepakh commented on a diff in pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
valepakh commented on code in PR #1024:
URL: https://github.com/apache/ignite-3/pull/1024#discussion_r954738272


##########
modules/cli/src/main/java/org/apache/ignite/cli/core/exception/handler/SqlExceptionHandler.java:
##########
@@ -60,13 +109,24 @@ public int handle(ExceptionWriter err, SQLException e) {
                 break;
             default:
                 LOG.error("Unrecognized error", e);
-                errorComponentBuilder.header("Unrecognized error while process SQL query");
+                errorComponentBuilder.header("SQL query execution error").details(e.getMessage());
         }
 
         err.write(errorComponentBuilder.build().render());
         return 1;
     }
 
+    /** Handles IgniteException that has more information like error code and trace id. */
+    public int handleIgniteException(ExceptionWriter err, IgniteException e) {
+        Function<IgniteException, ErrorComponentBuilder> errorComponentBuilder = sqlExceptionMappers.getOrDefault(
+                e.code(), defaultErrorMapper);

Review Comment:
   ```suggestion
           var errorComponentBuilder = sqlExceptionMappers.getOrDefault(e.code(), defaultErrorMapper);
   ```



##########
modules/cli/src/main/java/org/apache/ignite/cli/core/exception/handler/SqlExceptionHandler.java:
##########
@@ -39,8 +48,48 @@ public class SqlExceptionHandler implements ExceptionHandler<SQLException> {
 
     public static final String CONNECTION_BROKE_MESSAGE = "Connection error";
 
+    private final Map<Integer, Function<IgniteException, ErrorComponentBuilder>> sqlExceptionMappers = new HashMap<>();
+
+    private final Function<IgniteException, ErrorComponentBuilder> defaultErrorMapper =
+            (e) -> ErrorUiComponent.builder().header("Unrecognized error while process SQL query" + e.getMessage());
+
+    /** Default constructor. */
+    public SqlExceptionHandler() {
+        sqlExceptionMappers.put(Client.CONNECTION_ERR, this::connectionErrUiComponent);
+        sqlExceptionMappers.put(Sql.QUERY_INVALID_ERR, this::invalidQueryErrUiComponent);
+    }
+
+    private ErrorComponentBuilder invalidQueryErrUiComponent(IgniteException e) {
+        return ErrorUiComponent.builder()
+                .header(PARSING_ERROR_MESSAGE)
+                .errorCode(e.codeAsString())
+                .traceId(e.traceId())
+                .details(ErrorGroup.extractCauseMessage(e.getMessage()));
+    }
+
+    private ErrorComponentBuilder connectionErrUiComponent(IgniteException e) {
+        if (e.getCause() instanceof IgniteClientConnectionException) {
+            IgniteClientConnectionException clientConnectionException = (IgniteClientConnectionException) e.getCause();
+            return ErrorUiComponent.builder()
+                    .header(CLIENT_CONNECTION_FAILED_MESSAGE)
+                    .errorCode(clientConnectionException.codeAsString())
+                    .traceId(clientConnectionException.traceId())
+                    .details(ErrorGroup.extractCauseMessage(clientConnectionException.getMessage()));
+        }
+
+        return ErrorUiComponent.builder()
+                .header(CLIENT_CONNECTION_FAILED_MESSAGE)
+                .errorCode(e.codeAsString())
+                .traceId(e.traceId())
+                .details(ErrorGroup.extractCauseMessage(e.getMessage()));

Review Comment:
   Looks like there is a possibility to extract a method for IgniteCliException with custom header.



##########
modules/cli/src/main/java/org/apache/ignite/cli/core/exception/handler/SqlExceptionHandler.java:
##########
@@ -39,8 +48,48 @@ public class SqlExceptionHandler implements ExceptionHandler<SQLException> {
 
     public static final String CONNECTION_BROKE_MESSAGE = "Connection error";
 
+    private final Map<Integer, Function<IgniteException, ErrorComponentBuilder>> sqlExceptionMappers = new HashMap<>();
+
+    private final Function<IgniteException, ErrorComponentBuilder> defaultErrorMapper =
+            (e) -> ErrorUiComponent.builder().header("Unrecognized error while process SQL query" + e.getMessage());

Review Comment:
   Maybe use a method similar to invalidQueryErrUiComponent to extract code, traceId and cause message?
   Also "process"->"processing"



-- 
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] Pochatkin commented on a diff in pull request #1024: IGNITE-17090 Show user-friendly message in sql command

Posted by GitBox <gi...@apache.org>.
Pochatkin commented on code in PR #1024:
URL: https://github.com/apache/ignite-3/pull/1024#discussion_r965608862


##########
modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java:
##########
@@ -272,10 +273,14 @@ public CompletableFuture<JdbcMetaPrimaryKeysResult> primaryKeysMetaAsync(JdbcMet
      * @return StringWriter filled with exception.
      */
     private StringWriter getWriterWithStackTrace(Throwable t) {
+        String message = ExceptionUtils.unwrapCause(t).getMessage();
         StringWriter sw = new StringWriter();
         PrintWriter pw = new PrintWriter(sw);
 
-        t.printStackTrace(pw);
+        pw.print(message);
+
+        // check if stacktrace is needed
+        // t.printStackTrace(pw);

Review Comment:
   Done



-- 
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: notifications-unsubscribe@ignite.apache.org

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