You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by so...@apache.org on 2019/05/03 22:46:30 UTC

[drill] 04/08: DRILL-6988. Utility of the too long error message when syntax error

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

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

commit d897f709e60557be38deff2276118c8a0b0196c5
Author: Dmytriy Grinchenko <dm...@gmail.com>
AuthorDate: Wed Apr 17 12:57:39 2019 +0300

    DRILL-6988. Utility of the too long error message when syntax error
    
    - Adding Drill wrapper around SqlparseException to customize produced by Calcite messages
    - Fix Drill SQL parse exception formatter to calculate proper position for "^" character
    closes #1753
---
 .../drill/exec/planner/sql/SqlConverter.java       | 104 +++++++++++---------
 .../exec/planner/sql/parser/DrillParserUtil.java   |   5 +
 .../sql/parser/impl/DrillSqlParseException.java    | 105 +++++++++++++++++++++
 .../drill/exec/planner/sql/TestDrillSQLWorker.java |  33 ++++---
 4 files changed, 189 insertions(+), 58 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index 237b57b..17b0490 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -26,6 +26,10 @@ import java.util.Objects;
 import java.util.Properties;
 import java.util.Set;
 
+import org.apache.calcite.util.Static;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
+import org.apache.drill.exec.planner.sql.parser.impl.DrillSqlParseException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.MetadataProviderManager;
 import org.apache.drill.exec.physical.base.TableMetadataProvider;
@@ -59,6 +63,7 @@ import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
@@ -90,7 +95,6 @@ import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.store.dfs.FileSelection;
-import static org.apache.calcite.util.Static.RESOURCE;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
 import org.apache.drill.exec.store.ColumnExplorer;
@@ -106,6 +110,7 @@ public class SqlConverter {
 
   private final JavaTypeFactory typeFactory;
   private final SqlParser.Config parserConfig;
+
   // Allow the default config to be modified using immutable configs
   private SqlToRelConverter.Config sqlToRelConverterConfig;
   private final DrillCalciteCatalogReader catalog;
@@ -143,7 +148,7 @@ public class SqlConverter {
     this.sqlToRelConverterConfig = new SqlToRelConverterConfig();
     this.isInnerQuery = false;
     this.typeFactory = new JavaTypeFactoryImpl(DRILL_TYPE_SYSTEM);
-    this.defaultSchema =  context.getNewDefaultSchema();
+    this.defaultSchema = context.getNewDefaultSchema();
     this.rootSchema = rootSchema(defaultSchema);
     this.temporarySchema = context.getConfig().getString(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE);
     this.session = context.getSession();
@@ -191,15 +196,15 @@ public class SqlConverter {
       SqlParser parser = SqlParser.create(sql, parserConfig);
       return parser.parseStmt();
     } catch (SqlParseException e) {
+      DrillSqlParseException dex = new DrillSqlParseException(e);
       UserException.Builder builder = UserException
-          .parseError(e)
-          .addContext("SQL Query", formatSQLParsingError(sql, e.getPos()));
+          .parseError(dex)
+          .addContext(formatSQLParsingError(sql, dex));
       if (isInnerQuery) {
         builder.message("Failure parsing a view your query is dependent upon.");
       }
       throw builder.build(logger);
     }
-
   }
 
   public SqlNode validate(final SqlNode parsedNode) {
@@ -265,26 +270,25 @@ public class SqlConverter {
         SqlNode node,
         RelDataType targetRowType,
         SqlValidatorScope scope) {
-      switch (node.getKind()) {
-        case AS:
-          SqlNode sqlNode = ((SqlCall) node).operand(0);
-          switch (sqlNode.getKind()) {
-            case IDENTIFIER:
-              SqlIdentifier tempNode = (SqlIdentifier) sqlNode;
-              DrillCalciteCatalogReader catalogReader = (SqlConverter.DrillCalciteCatalogReader) getCatalogReader();
-
-              changeNamesIfTableIsTemporary(tempNode);
-
-              // Check the schema and throw a valid SchemaNotFound exception instead of TableNotFound exception.
-              if (catalogReader.getTable(tempNode.names) == null) {
-                catalogReader.isValidSchema(tempNode.names);
-              }
-              break;
-            case UNNEST:
-              if (((SqlCall) node).operandCount() < 3) {
-                throw RESOURCE.validationError("Alias table and column name are required for UNNEST").ex();
-              }
-          }
+      if (node.getKind() == SqlKind.AS) {
+        SqlNode sqlNode = ((SqlCall) node).operand(0);
+        switch (sqlNode.getKind()) {
+          case IDENTIFIER:
+            SqlIdentifier tempNode = (SqlIdentifier) sqlNode;
+            DrillCalciteCatalogReader catalogReader = (DrillCalciteCatalogReader) getCatalogReader();
+
+            changeNamesIfTableIsTemporary(tempNode);
+
+            // Check the schema and throw a valid SchemaNotFound exception instead of TableNotFound exception.
+            if (catalogReader.getTable(tempNode.names) == null) {
+              catalogReader.isValidSchema(tempNode.names);
+            }
+            break;
+          case UNNEST:
+            if (((SqlCall) node).operandCount() < 3) {
+              throw Static.RESOURCE.validationError("Alias table and column name are required for UNNEST").ex();
+            }
+        }
       }
       super.validateFrom(node, targetRowType, scope);
     }
@@ -493,30 +497,44 @@ public class SqlConverter {
   }
 
   /**
+   * Formats sql exception with context name included and with
+   * graphical representation for the {@link DrillSqlParseException}
    *
-   * @param sql
-   *          the SQL sent to the server
-   * @param pos
-   *          the position of the error
+   * @param sql     the SQL sent to the server
+   * @param ex      exception object
    * @return The sql with a ^ character under the error
    */
-  static String formatSQLParsingError(String sql, SqlParserPos pos) {
-    if (pos == null) {
-      return sql;
-    }
-    StringBuilder sb = new StringBuilder();
-    String[] lines = sql.split("\n");
-    for (int i = 0; i < lines.length; i++) {
-      String line = lines[i];
-      sb.append(line).append("\n");
-      if (i == (pos.getLineNum() - 1)) {
-        for (int j = 0; j < pos.getColumnNum() - 1; j++) {
-          sb.append(" ");
+  static String formatSQLParsingError(String sql, DrillSqlParseException ex) {
+    final String sqlErrorMessageHeader = "SQL Query: ";
+    final SqlParserPos pos = ex.getPos();
+
+    if (pos != null) {
+      int issueLineNumber = pos.getLineNum() - 1;  // recalculates to base 0
+      int issueColumnNumber = pos.getColumnNum() - 1;  // recalculates to base 0
+      int messageHeaderLength = sqlErrorMessageHeader.length();
+
+      // If the issue happens on the first line, header width should be calculated alongside with the sql query
+      int shiftLength = (issueLineNumber == 0) ? issueColumnNumber + messageHeaderLength : issueColumnNumber;
+
+      StringBuilder sb = new StringBuilder();
+      String[] lines = sql.split(DrillParserUtil.EOL);
+
+      for (int i = 0; i < lines.length; i++) {
+        sb.append(lines[i]);
+
+        if (i == issueLineNumber) {
+          sb
+              .append(DrillParserUtil.EOL)
+              .append(StringUtils.repeat(' ', shiftLength))
+              .append("^");
+        }
+        if (i < lines.length - 1) {
+          sb.append(DrillParserUtil.EOL);
         }
-        sb.append("^\n");
       }
+      sql = sb.toString();
     }
-    return sb.toString();
+    return sqlErrorMessageHeader + sql;
   }
 
   private static SchemaPlus rootSchema(SchemaPlus schema) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
index ee1106d..06e5881 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
@@ -32,6 +32,11 @@ public class DrillParserUtil {
 
   private static final int CONDITION_LIST_CAPACITY = 3;
 
+  /**
+   * System-depended end of line character
+   */
+  public static final String EOL = System.lineSeparator();
+
   public static SqlNode createCondition(SqlNode left, SqlOperator op, SqlNode right) {
 
     // if one of the operands is null, return the other
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillSqlParseException.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillSqlParseException.java
new file mode 100644
index 0000000..f248241
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillSqlParseException.java
@@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.planner.sql.parser.impl;
+
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+/**
+ * Customized {@link SqlParseException} class
+ */
+public class DrillSqlParseException extends SqlParseException {
+  private final ParseException parseException;
+
+  public DrillSqlParseException(String message, SqlParserPos pos, int[][] expectedTokenSequences,
+                                String[] tokenImages, Throwable ex) {
+    super(message, pos, expectedTokenSequences, tokenImages, ex);
+
+    parseException = (ex instanceof ParseException) ? (ParseException) ex : null;
+  }
+
+  public DrillSqlParseException(SqlParseException sqlParseException) {
+    this(sqlParseException.getMessage(), sqlParseException.getPos(), sqlParseException.getExpectedTokenSequences(),
+        sqlParseException.getTokenImages(), sqlParseException.getCause());
+  }
+
+  /**
+   * Builds error message just like the original {@link SqlParseException}
+   * with special handling for {@link ParseException} class.
+   * <p>
+   * This is customized implementation of the original {@link ParseException#getMessage()}.
+   * Any other underlying {@link SqlParseException} exception messages goes through without changes
+   * <p>
+   * <p>
+   * Example:
+   * <pre>
+   *
+   *   Given query: SELECT FROM (VALUES(1));
+   *
+   *   Generated message for the unsupported FROM token after SELECT would look like:
+   *
+   *       Encountered "FROM" at line 1, column 8.
+   *</pre>
+   * @return formatted string representation of {@link DrillSqlParseException}
+   */
+  @Override
+  public String getMessage() {
+    // proxy the original message if exception does not belongs
+    // to ParseException or no current token passed
+    if (parseException == null || parseException.currentToken == null) {
+      return super.getMessage();
+    }
+
+    int[][] expectedTokenSequences = getExpectedTokenSequences();
+    String[] tokenImage = getTokenImages();
+
+    int maxSize = 0;  // holds max possible length of the token sequence
+    for (int[] expectedTokenSequence : expectedTokenSequences) {
+      if (maxSize < expectedTokenSequence.length) {
+        maxSize = expectedTokenSequence.length;
+      }
+    }
+
+    // parseException.currentToken points to the last successfully parsed token, next one is considered as fail reason
+    Token tok = parseException.currentToken.next;
+    StringBuilder sb = new StringBuilder("Encountered \"");
+
+    // Adds unknown token sequences to the message
+    for (int i = 0; i < maxSize; i++) {
+      if (i != 0) {
+        sb.append(" ");
+      }
+
+      if (tok.kind == DrillParserImplConstants.EOF) {
+        sb.append(tokenImage[0]);
+        break;
+      }
+      sb.append(parseException.add_escapes(tok.image));
+      tok = tok.next;
+    }
+
+    sb
+        .append("\" at line ")
+        .append(parseException.currentToken.beginLine)
+        .append(", column ")
+        .append(parseException.currentToken.next.beginColumn)
+        .append(".");
+
+    return sb.toString();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java
index 57c5de5..941cdfc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/sql/TestDrillSQLWorker.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 
 import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.drill.exec.planner.sql.parser.impl.DrillSqlParseException;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SqlTest;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
@@ -31,28 +32,30 @@ import org.junit.experimental.categories.Category;
 public class TestDrillSQLWorker extends BaseTestQuery {
 
   private void validateFormattedIs(String sql, SqlParserPos pos, String expected) {
-    String formatted = SqlConverter.formatSQLParsingError(sql, pos);
+    DrillSqlParseException ex = new DrillSqlParseException(null, pos, null, null, null);
+    String formatted = SqlConverter.formatSQLParsingError(sql, ex);
     assertEquals(expected, formatted);
   }
 
   @Test
-  public void testErrorFormating() {
-    String sql = "Select * from Foo\nwhere tadadidada;\n";
+  public void testErrorFormatting() {
+    String sql = "Select * from Foo\n"
+        + "where tadadidada;\n";
     validateFormattedIs(sql, new SqlParserPos(1, 2),
-        "Select * from Foo\n"
-      + " ^\n"
-      + "where tadadidada;\n");
+        "SQL Query: Select * from Foo\n"
+            + "            ^\n"
+            + "where tadadidada;");
     validateFormattedIs(sql, new SqlParserPos(2, 2),
-        "Select * from Foo\n"
-      + "where tadadidada;\n"
-      + " ^\n" );
+        "SQL Query: Select * from Foo\n"
+            + "where tadadidada;\n"
+            + " ^" );
     validateFormattedIs(sql, new SqlParserPos(1, 10),
-        "Select * from Foo\n"
-      + "         ^\n"
-      + "where tadadidada;\n");
-    validateFormattedIs(sql, new SqlParserPos(-11, -10), sql);
-    validateFormattedIs(sql, new SqlParserPos(0, 10), sql);
-    validateFormattedIs(sql, new SqlParserPos(100, 10), sql);
+        "SQL Query: Select * from Foo\n"
+            + "                    ^\n"
+            + "where tadadidada;");
+    validateFormattedIs(sql, new SqlParserPos(-11, -10), "SQL Query: Select * from Foo\nwhere tadadidada;");
+    validateFormattedIs(sql, new SqlParserPos(0, 10), "SQL Query: Select * from Foo\nwhere tadadidada;");
+    validateFormattedIs(sql, new SqlParserPos(100, 10), "SQL Query: Select * from Foo\nwhere tadadidada;");
   }
 
   @Test