You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/09/16 20:56:30 UTC

[GitHub] [spark] karenfeng commented on a change in pull request #33535: [SPARK-36108][SQL] Refactor first set of 20 query parsing errors to use error classes

karenfeng commented on a change in pull request #33535:
URL: https://github.com/apache/spark/pull/33535#discussion_r710451185



##########
File path: core/src/main/resources/error/error-classes.json
##########
@@ -57,9 +93,25 @@
     "message" : [ "The second argument of '%s' function needs to be an integer." ],
     "sqlState" : "22023"
   },
+  "TRANSFORM_NOT_SUPPORT_QUANTIFIER" : {
+    "message" : [ "TRANSFORM does not support DISTINCT/ALL in inputs" ],
+    "sqlState" : "42000"
+  },
+  "TRANSFORM_WITH_SERDE_UNSUPPORTED" : {
+    "message" : [ "TRANSFORM with serde is only supported in hive mode" ],
+    "sqlState" : "42000"

Review comment:
       This should probably be 0A000

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -29,91 +29,84 @@ import org.apache.spark.sql.catalyst.trees.Origin
  */
 object QueryParsingErrors {
 
-  def invalidInsertIntoError(ctx: InsertIntoContext): Throwable = {
-    new ParseException("Invalid InsertIntoContext", ctx)
-  }
-
   def insertOverwriteDirectoryUnsupportedError(ctx: InsertIntoContext): Throwable = {
-    new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx)
+    new ParseException("OPERATION_UNSUPPORTED", Array("INSERT OVERWRITE DIRECTORY"), ctx)
   }
 
   def columnAliasInOperationNotAllowedError(op: String, ctx: TableAliasContext): Throwable = {
-    new ParseException(s"Columns aliases are not allowed in $op.", ctx.identifierList())
+    new ParseException("COLUMN_ALIASES_NOT_ALLOWED_IN_OPERATION", Array(op), ctx.identifierList())
   }
 
   def emptySourceForMergeError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("Empty source for merge: you should specify a source" +
-      " table/subquery in merge.", ctx.source)
+    new ParseException("EMPTY_SOURCE_FOR_MERGE", Array.empty, ctx.source)
   }
 
   def unrecognizedMatchedActionError(ctx: MatchedClauseContext): Throwable = {
-    new ParseException(s"Unrecognized matched action: ${ctx.matchedAction().getText}",
-      ctx.matchedAction())
+    new ParseException("UNRECOGNIZED_ACTION",
+      Array("matched", ctx.matchedAction().getText), ctx.matchedAction())
   }
 
   def insertedValueNumberNotMatchFieldNumberError(ctx: NotMatchedClauseContext): Throwable = {
-    new ParseException("The number of inserted values cannot match the fields.",
+    new ParseException("INSERTED_VALUE_NUMBER_NOT_MATCH_FIELD_NUMBER", Array.empty,
       ctx.notMatchedAction())
   }
 
   def unrecognizedNotMatchedActionError(ctx: NotMatchedClauseContext): Throwable = {
-    new ParseException(s"Unrecognized not matched action: ${ctx.notMatchedAction().getText}",
-      ctx.notMatchedAction())
+    new ParseException("UNRECOGNIZED_ACTION",
+      Array("not matched", ctx.notMatchedAction().getText), ctx.notMatchedAction())

Review comment:
       Can we make `not matched` all caps? This may make it clearer.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
##########
@@ -292,8 +292,6 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg
       case hiveDir: InsertOverwriteHiveDirContext =>
         val (isLocal, storage, provider) = visitInsertOverwriteHiveDir(hiveDir)
         InsertIntoDir(isLocal, storage, provider, query, overwrite = true)
-      case _ =>

Review comment:
       Why was this removed?

##########
File path: core/src/main/resources/error/error-classes.json
##########
@@ -57,9 +93,25 @@
     "message" : [ "The second argument of '%s' function needs to be an integer." ],
     "sqlState" : "22023"
   },
+  "TRANSFORM_NOT_SUPPORT_QUANTIFIER" : {
+    "message" : [ "TRANSFORM does not support DISTINCT/ALL in inputs" ],
+    "sqlState" : "42000"
+  },
+  "TRANSFORM_WITH_SERDE_UNSUPPORTED" : {
+    "message" : [ "TRANSFORM with serde is only supported in hive mode" ],
+    "sqlState" : "42000"
+  },
   "UNABLE_TO_ACQUIRE_MEMORY" : {
     "message" : [ "Unable to acquire %s bytes of memory, got %s" ]
   },
+  "UNRECOGNIZED_ACTION" : {
+    "message" : [ "Unrecognized %s action: %s" ],
+    "sqlState" : "42000"
+  },
+  "UNSUPPORTED_LATERAL_JOIN_TYPE" : {
+    "message" : [ "Unsupported LATERAL join type %s" ],
+    "sqlState" : "42000"
+  },

Review comment:
       This should probably be 0A000

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -29,91 +29,84 @@ import org.apache.spark.sql.catalyst.trees.Origin
  */
 object QueryParsingErrors {
 
-  def invalidInsertIntoError(ctx: InsertIntoContext): Throwable = {
-    new ParseException("Invalid InsertIntoContext", ctx)
-  }
-
   def insertOverwriteDirectoryUnsupportedError(ctx: InsertIntoContext): Throwable = {
-    new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx)
+    new ParseException("OPERATION_UNSUPPORTED", Array("INSERT OVERWRITE DIRECTORY"), ctx)
   }
 
   def columnAliasInOperationNotAllowedError(op: String, ctx: TableAliasContext): Throwable = {
-    new ParseException(s"Columns aliases are not allowed in $op.", ctx.identifierList())
+    new ParseException("COLUMN_ALIASES_NOT_ALLOWED_IN_OPERATION", Array(op), ctx.identifierList())
   }
 
   def emptySourceForMergeError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("Empty source for merge: you should specify a source" +
-      " table/subquery in merge.", ctx.source)
+    new ParseException("EMPTY_SOURCE_FOR_MERGE", Array.empty, ctx.source)
   }
 
   def unrecognizedMatchedActionError(ctx: MatchedClauseContext): Throwable = {
-    new ParseException(s"Unrecognized matched action: ${ctx.matchedAction().getText}",
-      ctx.matchedAction())
+    new ParseException("UNRECOGNIZED_ACTION",
+      Array("matched", ctx.matchedAction().getText), ctx.matchedAction())

Review comment:
       Can we make `matched` all caps? This may make it clearer.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -29,91 +29,84 @@ import org.apache.spark.sql.catalyst.trees.Origin
  */
 object QueryParsingErrors {
 
-  def invalidInsertIntoError(ctx: InsertIntoContext): Throwable = {
-    new ParseException("Invalid InsertIntoContext", ctx)
-  }
-
   def insertOverwriteDirectoryUnsupportedError(ctx: InsertIntoContext): Throwable = {
-    new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx)
+    new ParseException("OPERATION_UNSUPPORTED", Array("INSERT OVERWRITE DIRECTORY"), ctx)
   }
 
   def columnAliasInOperationNotAllowedError(op: String, ctx: TableAliasContext): Throwable = {
-    new ParseException(s"Columns aliases are not allowed in $op.", ctx.identifierList())
+    new ParseException("COLUMN_ALIASES_NOT_ALLOWED_IN_OPERATION", Array(op), ctx.identifierList())
   }
 
   def emptySourceForMergeError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("Empty source for merge: you should specify a source" +
-      " table/subquery in merge.", ctx.source)
+    new ParseException("EMPTY_SOURCE_FOR_MERGE", Array.empty, ctx.source)
   }
 
   def unrecognizedMatchedActionError(ctx: MatchedClauseContext): Throwable = {
-    new ParseException(s"Unrecognized matched action: ${ctx.matchedAction().getText}",
-      ctx.matchedAction())
+    new ParseException("UNRECOGNIZED_ACTION",
+      Array("matched", ctx.matchedAction().getText), ctx.matchedAction())
   }
 
   def insertedValueNumberNotMatchFieldNumberError(ctx: NotMatchedClauseContext): Throwable = {
-    new ParseException("The number of inserted values cannot match the fields.",
+    new ParseException("INSERTED_VALUE_NUMBER_NOT_MATCH_FIELD_NUMBER", Array.empty,
       ctx.notMatchedAction())
   }
 
   def unrecognizedNotMatchedActionError(ctx: NotMatchedClauseContext): Throwable = {
-    new ParseException(s"Unrecognized not matched action: ${ctx.notMatchedAction().getText}",
-      ctx.notMatchedAction())
+    new ParseException("UNRECOGNIZED_ACTION",
+      Array("not matched", ctx.notMatchedAction().getText), ctx.notMatchedAction())
   }
 
   def mergeStatementWithoutWhenClauseError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("There must be at least one WHEN clause in a MERGE statement", ctx)
+    new ParseException("MERGE_STATEMENT_WITHOUT_WHEN_CLAUSE", Array.empty, ctx)
   }
 
   def nonLastMatchedClauseOmitConditionError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("When there are more than one MATCHED clauses in a MERGE " +
-      "statement, only the last MATCHED clause can omit the condition.", ctx)
+    new ParseException("NON_LAST_CLAUSE_OMIT_CONDITION", Array("MATCHED", "MATCHED"), ctx)
   }
 
   def nonLastNotMatchedClauseOmitConditionError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("When there are more than one NOT MATCHED clauses in a MERGE " +
-      "statement, only the last NOT MATCHED clause can omit the condition.", ctx)
+    new ParseException("NON_LAST_CLAUSE_OMIT_CONDITION", Array("NOT MATCHED", "NOT MATCHED"), ctx)
   }
 
   def emptyPartitionKeyError(key: String, ctx: PartitionSpecContext): Throwable = {
-    new ParseException(s"Found an empty partition key '$key'.", ctx)
+    new ParseException("EMPTY_PARTITION_KEY", Array(key), ctx)
   }
 
   def combinationQueryResultClausesUnsupportedError(ctx: QueryOrganizationContext): Throwable = {
-    new ParseException(
-      "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
+    new ParseException("OPERATION_UNSUPPORTED",

Review comment:
       We should probably have a separate error class for unsupported combinations.

##########
File path: core/src/main/resources/error/error-classes.json
##########
@@ -29,10 +41,26 @@
     "message" : [ "Invalid pivot column '%s'. Pivot columns must be comparable." ],
     "sqlState" : "42000"
   },
+  "INSERTED_VALUE_NUMBER_NOT_MATCH_FIELD_NUMBER" : {
+    "message" : [ "The number of inserted values cannot match the fields." ],

Review comment:
       Nit: `cannot` -> `does not`

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
##########
@@ -42,6 +42,12 @@ class PlanParserSuite extends AnalysisTest {
   private def intercept(sqlCommand: String, messages: String*): Unit =
     interceptParseException(parsePlan)(sqlCommand, messages: _*)
 
+  private def interceptWithErrorClass(sqlCommand: String, messages: String*)(

Review comment:
       Nice test!

##########
File path: core/src/main/resources/error/error-classes.json
##########
@@ -29,10 +41,26 @@
     "message" : [ "Invalid pivot column '%s'. Pivot columns must be comparable." ],
     "sqlState" : "42000"
   },
+  "INSERTED_VALUE_NUMBER_NOT_MATCH_FIELD_NUMBER" : {

Review comment:
       The grammar is a bit strange. Maybe `INSERTED_VALUE_AND_FIELD_NUMBER_MISMATCH`?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
##########
@@ -29,91 +29,84 @@ import org.apache.spark.sql.catalyst.trees.Origin
  */
 object QueryParsingErrors {
 
-  def invalidInsertIntoError(ctx: InsertIntoContext): Throwable = {
-    new ParseException("Invalid InsertIntoContext", ctx)
-  }
-
   def insertOverwriteDirectoryUnsupportedError(ctx: InsertIntoContext): Throwable = {
-    new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx)
+    new ParseException("OPERATION_UNSUPPORTED", Array("INSERT OVERWRITE DIRECTORY"), ctx)
   }
 
   def columnAliasInOperationNotAllowedError(op: String, ctx: TableAliasContext): Throwable = {
-    new ParseException(s"Columns aliases are not allowed in $op.", ctx.identifierList())
+    new ParseException("COLUMN_ALIASES_NOT_ALLOWED_IN_OPERATION", Array(op), ctx.identifierList())
   }
 
   def emptySourceForMergeError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("Empty source for merge: you should specify a source" +
-      " table/subquery in merge.", ctx.source)
+    new ParseException("EMPTY_SOURCE_FOR_MERGE", Array.empty, ctx.source)
   }
 
   def unrecognizedMatchedActionError(ctx: MatchedClauseContext): Throwable = {
-    new ParseException(s"Unrecognized matched action: ${ctx.matchedAction().getText}",
-      ctx.matchedAction())
+    new ParseException("UNRECOGNIZED_ACTION",
+      Array("matched", ctx.matchedAction().getText), ctx.matchedAction())
   }
 
   def insertedValueNumberNotMatchFieldNumberError(ctx: NotMatchedClauseContext): Throwable = {
-    new ParseException("The number of inserted values cannot match the fields.",
+    new ParseException("INSERTED_VALUE_NUMBER_NOT_MATCH_FIELD_NUMBER", Array.empty,
       ctx.notMatchedAction())
   }
 
   def unrecognizedNotMatchedActionError(ctx: NotMatchedClauseContext): Throwable = {
-    new ParseException(s"Unrecognized not matched action: ${ctx.notMatchedAction().getText}",
-      ctx.notMatchedAction())
+    new ParseException("UNRECOGNIZED_ACTION",
+      Array("not matched", ctx.notMatchedAction().getText), ctx.notMatchedAction())
   }
 
   def mergeStatementWithoutWhenClauseError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("There must be at least one WHEN clause in a MERGE statement", ctx)
+    new ParseException("MERGE_STATEMENT_WITHOUT_WHEN_CLAUSE", Array.empty, ctx)
   }
 
   def nonLastMatchedClauseOmitConditionError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("When there are more than one MATCHED clauses in a MERGE " +
-      "statement, only the last MATCHED clause can omit the condition.", ctx)
+    new ParseException("NON_LAST_CLAUSE_OMIT_CONDITION", Array("MATCHED", "MATCHED"), ctx)
   }
 
   def nonLastNotMatchedClauseOmitConditionError(ctx: MergeIntoTableContext): Throwable = {
-    new ParseException("When there are more than one NOT MATCHED clauses in a MERGE " +
-      "statement, only the last NOT MATCHED clause can omit the condition.", ctx)
+    new ParseException("NON_LAST_CLAUSE_OMIT_CONDITION", Array("NOT MATCHED", "NOT MATCHED"), ctx)
   }
 
   def emptyPartitionKeyError(key: String, ctx: PartitionSpecContext): Throwable = {
-    new ParseException(s"Found an empty partition key '$key'.", ctx)
+    new ParseException("EMPTY_PARTITION_KEY", Array(key), ctx)
   }
 
   def combinationQueryResultClausesUnsupportedError(ctx: QueryOrganizationContext): Throwable = {
-    new ParseException(
-      "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
+    new ParseException("OPERATION_UNSUPPORTED",
+      Array("Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY"), ctx)
   }
 
   def distributeByUnsupportedError(ctx: QueryOrganizationContext): Throwable = {
-    new ParseException("DISTRIBUTE BY is not supported", ctx)
+    new ParseException("OPERATION_UNSUPPORTED", Array("DISTRIBUTE BY"), ctx)
   }
 
   def transformNotSupportQuantifierError(ctx: ParserRuleContext): Throwable = {
-    new ParseException("TRANSFORM does not support DISTINCT/ALL in inputs", ctx)
+    new ParseException("TRANSFORM_NOT_SUPPORT_QUANTIFIER", Array.empty, ctx)
   }
 
   def transformWithSerdeUnsupportedError(ctx: ParserRuleContext): Throwable = {
-    new ParseException("TRANSFORM with serde is only supported in hive mode", ctx)
+    new ParseException("TRANSFORM_WITH_SERDE_UNSUPPORTED", Array.empty, ctx)
   }
 
   def lateralWithPivotInFromClauseNotAllowedError(ctx: FromClauseContext): Throwable = {
-    new ParseException("LATERAL cannot be used together with PIVOT in FROM clause", ctx)
+    new ParseException("LATERAL_WITH_PIVOT_IN_FROM_CLAUSE_NOT_ALLOWED", Array.empty, ctx)
   }
 
   def lateralJoinWithNaturalJoinUnsupportedError(ctx: ParserRuleContext): Throwable = {
-    new ParseException("LATERAL join with NATURAL join is not supported", ctx)
+    new ParseException("OPERATION_UNSUPPORTED", Array("LATERAL join with NATURAL join"), ctx)

Review comment:
       I think we can also make a separate error class here as well, maybe about how different join types are incompatible.




-- 
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: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org