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 2022/11/15 07:40:50 UTC

[GitHub] [spark] itholic opened a new pull request, #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

itholic opened a new pull request, #38664:
URL: https://github.com/apache/spark/pull/38664

   
   ### What changes were proposed in this pull request?
   
   This PR proposes to assign a name to `_LEGACY_ERROR_TEMP_1042` as `INVALID_FUNCTION_ARGUMENT`.
   
   ### Why are the changes needed?
   
   The legacy error classes were temporary named to cover the all exceptions by error class, so we should assign the proper name on those all legacy error classes.
   
   ### Does this PR introduce _any_ user-facing change?
   
   No.
   
   ### How was this patch tested?
   
   ```
   ./build/sbt “sql/testOnly org.apache.spark.sql.SQLQueryTestSuite*”
   ```


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


[GitHub] [spark] itholic commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
itholic commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1028813303


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -146,7 +147,10 @@ object FunctionRegistryBase {
             .filter(_.getParameterTypes.forall(_ == classOf[Expression]))
             .map(_.getParameterCount).distinct.sorted
           throw QueryCompilationErrors.invalidFunctionArgumentNumberError(
-            validParametersCount, name, params.length)
+            expressions.map(toPrettySQL(_)).mkString(","),

Review Comment:
   Sounds good! Just addressed it.



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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1030185866


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -637,17 +637,24 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def invalidFunctionArgumentsError(
-      name: String, expectedInfo: String, actualNumber: Int): Throwable = {
+      sqlExpr: Seq[Expression],
+      name: String,
+      expectedInfo: String,
+      actualNumber: Int): Throwable = {
     new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1042",
+      errorClass = "DATATYPE_MISMATCH.WRONG_NUM_ARGS",
       messageParameters = Map(
-        "name" -> name,
-        "expectedInfo" -> expectedInfo,
-        "actualNumber" -> actualNumber.toString))
+        "sqlExpr" -> sqlExpr.map(toPrettySQL(_)).mkString(","),
+        "functionName" -> name,

Review Comment:
   Add quoting by `toSQLId()`



##########
sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out:
##########
@@ -791,11 +791,12 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "errorClass" : "DATATYPE_MISMATCH.WRONG_NUM_ARGS",
   "messageParameters" : {
-    "actualNumber" : "0",
-    "expectedInfo" : "2",
-    "name" : "decode"
+    "actualNum" : "0",
+    "expectedNum" : "2",
+    "functionName" : "decode",

Review Comment:
   The function name should be quoted.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -637,17 +637,24 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def invalidFunctionArgumentsError(
-      name: String, expectedInfo: String, actualNumber: Int): Throwable = {
+      sqlExpr: Seq[Expression],
+      name: String,
+      expectedInfo: String,
+      actualNumber: Int): Throwable = {
     new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1042",
+      errorClass = "DATATYPE_MISMATCH.WRONG_NUM_ARGS",
       messageParameters = Map(
-        "name" -> name,
-        "expectedInfo" -> expectedInfo,
-        "actualNumber" -> actualNumber.toString))
+        "sqlExpr" -> sqlExpr.map(toPrettySQL(_)).mkString(","),

Review Comment:
   Could you invoke `toSQLExpr()` instead of `toPrettySQL()`, please. Take a look at the other exceptions in the file.



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


[GitHub] [spark] itholic commented on pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
itholic commented on PR #38664:
URL: https://github.com/apache/spark/pull/38664#issuecomment-1326044896

   Thanks for the review, @MaxGekk 
   Just addressed the comments!


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


[GitHub] [spark] itholic closed pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
itholic closed pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`
URL: https://github.com/apache/spark/pull/38664


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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1032047234


##########
sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out:
##########
@@ -848,7 +849,8 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "actualNum" : "1",
     "expectedNum" : "2",
-    "functionName" : "`decode`"
+    "functionName" : "`decode`",
+    "sqlExpr" : "\"encode(abc, utf-8)\""

Review Comment:
   The error message confuses slightly:
   ```
   "Cannot resolve "encode(abc, utf-8)" due to data type mismatch:"
   ```
   Actually, the issue is not in `encode` but in `decode`. `sqlExpr` should point out to the parent expression. @itholic Could you change this?



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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1024815444


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -146,7 +146,7 @@ object FunctionRegistryBase {
             .filter(_.getParameterTypes.forall(_ == classOf[Expression]))
             .map(_.getParameterCount).distinct.sorted
           throw QueryCompilationErrors.invalidFunctionArgumentNumberError(
-            validParametersCount, name, params.length)
+            expressions.map(_.sql).mkString(","), validParametersCount, name, params.length)

Review Comment:
   Invoke `toSQLExpr` instead of `sql()`



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


[GitHub] [spark] itholic commented on pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
itholic commented on PR #38664:
URL: https://github.com/apache/spark/pull/38664#issuecomment-1315140725

   Sure, let me integrate into `DATATYPE_MISMATCH.WRONG_NUM_ARGS`


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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1026264084


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala:
##########
@@ -146,7 +147,10 @@ object FunctionRegistryBase {
             .filter(_.getParameterTypes.forall(_ == classOf[Expression]))
             .map(_.getParameterCount).distinct.sorted
           throw QueryCompilationErrors.invalidFunctionArgumentNumberError(
-            validParametersCount, name, params.length)
+            expressions.map(toPrettySQL(_)).mkString(","),

Review Comment:
   Could you pass `Seq[Expression]` into `invalidFunctionArgumentNumberError()`, and form a string inside of the method as we do in other places like:
   ```scala
     def moreThanOneGeneratorError(generators: Seq[Expression], clause: String): Throwable = {
       new AnalysisException(
         errorClass = "UNSUPPORTED_GENERATOR.MULTI_GENERATOR",
         messageParameters = Map(
           "clause" -> clause,
           "num" -> generators.size.toString,
           "generators" -> generators.map(toSQLExpr).mkString(", ")))
     }
   ```
   
   For instance, if we change the implementation of `toSQLExpr()`, your exceptions won't pick up new impl.



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


[GitHub] [spark] itholic commented on pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
itholic commented on PR #38664:
URL: https://github.com/apache/spark/pull/38664#issuecomment-1341896379

   Closing since it's duplicated to #38707 


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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1040017671


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -637,17 +637,21 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def invalidFunctionArgumentsError(
-      name: String, expectedNum: String, actualNum: Int): Throwable = {
+      sqlExpr: Seq[Expression], name: String, expectedNum: String, actualNum: Int): Throwable = {
     new AnalysisException(
       errorClass = "WRONG_NUM_ARGS",
       messageParameters = Map(
+        "sqlExpr" -> sqlExpr.map(toSQLExpr(_)).mkString(","),

Review Comment:
   This is not needed anymore. Could you remove it, please.



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


[GitHub] [spark] itholic commented on pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
itholic commented on PR #38664:
URL: https://github.com/apache/spark/pull/38664#issuecomment-1341896519

   Closing since it's duplicated to #38707 


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


[GitHub] [spark] itholic commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
itholic commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1034604147


##########
sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out:
##########
@@ -848,7 +849,8 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "actualNum" : "1",
     "expectedNum" : "2",
-    "functionName" : "`decode`"
+    "functionName" : "`decode`",
+    "sqlExpr" : "\"encode(abc, utf-8)\""

Review Comment:
   Let me handle it. Thanks!



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


[GitHub] [spark] itholic commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
itholic commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1040452617


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -637,17 +637,21 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def invalidFunctionArgumentsError(
-      name: String, expectedNum: String, actualNum: Int): Throwable = {
+      sqlExpr: Seq[Expression], name: String, expectedNum: String, actualNum: Int): Throwable = {
     new AnalysisException(
       errorClass = "WRONG_NUM_ARGS",
       messageParameters = Map(
+        "sqlExpr" -> sqlExpr.map(toSQLExpr(_)).mkString(","),

Review Comment:
   Btw, seems like it's duplicated to https://github.com/apache/spark/pull/38707 ?
   
   Can we just close this ticket ?



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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38664: [SPARK-41147][SQL] Assign a name to the legacy error class `_LEGACY_ERROR_TEMP_1042`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38664:
URL: https://github.com/apache/spark/pull/38664#discussion_r1035862190


##########
sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out:
##########
@@ -848,7 +849,8 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "actualNum" : "1",
     "expectedNum" : "2",
-    "functionName" : "`decode`"
+    "functionName" : "`decode`",
+    "sqlExpr" : "\"encode(abc, utf-8)\""

Review Comment:
   ok. Ping me when you finish this.



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