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/05/20 09:29:38 UTC

[GitHub] [spark] panbingkun opened a new pull request, #36617: [SPARK-38687][SQL] Use error classes in the compilation errors of generators

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

   ## What changes were proposed in this pull request?
   Migrate the following errors in QueryCompilationErrors onto use error classes:
   
   - nestedGeneratorError => UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS
   - moreThanOneGeneratorError => UNSUPPORTED_GENERATOR.MULTI_GENERATOR
   - generatorOutsideSelectError => UNSUPPORTED_GENERATOR.OUTSIDE_SELECT
   - generatorNotExpectedError => UNSUPPORTED_GENERATOR.NOT_GENERATOR
   
   ### Why are the changes needed?
   Porting compilation errors of generator to new error framework, improve test coverage, and document expected error messages in tests.
   
   ### Does this PR introduce any user-facing change?
   No
   
   ### How was this patch tested?
   By running new test:
   ```
   $ build/sbt "sql/testOnly *QueryCompilationErrorsSuite*"
   ```


-- 
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 pull request #36617: [SPARK-38687][SQL] Use error classes in the compilation errors of generators

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

   @panbingkun Could you backport the changes to branch-3.3, 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] MaxGekk closed pull request #36617: [SPARK-38687][SQL] Use error classes in the compilation errors of generators

Posted by GitBox <gi...@apache.org>.
MaxGekk closed pull request #36617: [SPARK-38687][SQL] Use error classes in the compilation errors of generators
URL: https://github.com/apache/spark/pull/36617


-- 
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] panbingkun commented on pull request #36617: [SPARK-38687][SQL] Use error classes in the compilation errors of generators

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

   > @panbingkun Could you backport the changes to branch-3.3, please.
   
   ok, i will do 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 #36617: [SPARK-38687][SQL] Use error classes in the compilation errors of generators

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


##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -583,6 +583,66 @@ class QueryCompilationErrorsSuite
       msg = "The deserializer is not supported: try to map \"STRUCT<a: STRING, b: INT>\" " +
         "to Tuple1, but failed as the number of fields does not line up.")
   }
+
+  test("UNSUPPORTED_GENERATOR: " +
+    "generators are not supported when it's nested in expressions") {
+    val e = intercept[AnalysisException](
+      sql("""select explode(Array(1, 2, 3)) + 1""").collect()
+    )
+
+    checkErrorClass(
+      exception = e,
+      errorClass = "UNSUPPORTED_GENERATOR",
+      errorSubClass = Some("NESTED_IN_EXPRESSIONS"),
+      msg = "The generator is not supported: nested in expressions (explode(array(1, 2, 3)) + 1)")

Review Comment:
   Please, quote (explode(array(1, 2, 3)) + 1) by default (double quotes). Maybe, it is better to add a helper method `toSQLExpr()` to QueryErrorsBase



##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -583,6 +583,66 @@ class QueryCompilationErrorsSuite
       msg = "The deserializer is not supported: try to map \"STRUCT<a: STRING, b: INT>\" " +
         "to Tuple1, but failed as the number of fields does not line up.")
   }
+
+  test("UNSUPPORTED_GENERATOR: " +
+    "generators are not supported when it's nested in expressions") {
+    val e = intercept[AnalysisException](
+      sql("""select explode(Array(1, 2, 3)) + 1""").collect()
+    )
+
+    checkErrorClass(
+      exception = e,
+      errorClass = "UNSUPPORTED_GENERATOR",
+      errorSubClass = Some("NESTED_IN_EXPRESSIONS"),
+      msg = "The generator is not supported: nested in expressions (explode(array(1, 2, 3)) + 1)")
+  }
+
+  test("UNSUPPORTED_GENERATOR: only one generator allowed") {
+    val e = intercept[AnalysisException](
+      sql("""select explode(Array(1, 2, 3)), explode(Array(1, 2, 3))""").collect()
+    )
+
+    checkErrorClass(
+      exception = e,
+      errorClass = "UNSUPPORTED_GENERATOR",
+      errorSubClass = Some("MULTI_GENERATOR"),
+      msg = "The generator is not supported: only one generator allowed per select clause " +
+        "but found 2: explode(array(1, 2, 3)), explode(array(1, 2, 3))"

Review Comment:
   Let quote expressions:
   explode(array(1, 2, 3)), explode(array(1, 2, 3)) -> "explode(array(1, 2, 3))", "explode(array(1, 2, 3))"



##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala:
##########
@@ -583,6 +583,66 @@ class QueryCompilationErrorsSuite
       msg = "The deserializer is not supported: try to map \"STRUCT<a: STRING, b: INT>\" " +
         "to Tuple1, but failed as the number of fields does not line up.")
   }
+
+  test("UNSUPPORTED_GENERATOR: " +
+    "generators are not supported when it's nested in expressions") {
+    val e = intercept[AnalysisException](
+      sql("""select explode(Array(1, 2, 3)) + 1""").collect()
+    )
+
+    checkErrorClass(
+      exception = e,
+      errorClass = "UNSUPPORTED_GENERATOR",
+      errorSubClass = Some("NESTED_IN_EXPRESSIONS"),
+      msg = "The generator is not supported: nested in expressions (explode(array(1, 2, 3)) + 1)")
+  }
+
+  test("UNSUPPORTED_GENERATOR: only one generator allowed") {
+    val e = intercept[AnalysisException](
+      sql("""select explode(Array(1, 2, 3)), explode(Array(1, 2, 3))""").collect()
+    )
+
+    checkErrorClass(
+      exception = e,
+      errorClass = "UNSUPPORTED_GENERATOR",
+      errorSubClass = Some("MULTI_GENERATOR"),
+      msg = "The generator is not supported: only one generator allowed per select clause " +
+        "but found 2: explode(array(1, 2, 3)), explode(array(1, 2, 3))"
+    )
+  }
+
+  test("UNSUPPORTED_GENERATOR: generators are not supported outside the SELECT clause") {
+    val e = intercept[AnalysisException](
+      sql("""select 1 from t order by explode(Array(1, 2, 3))""").collect()
+    )
+
+    checkErrorClass(
+      exception = e,
+      errorClass = "UNSUPPORTED_GENERATOR",
+      errorSubClass = Some("OUTSIDE_SELECT"),
+      msg = "The generator is not supported: outside the SELECT clause, found: " +
+        "'Sort [explode(array(1, 2, 3)) ASC NULLS FIRST], true"
+    )
+  }
+
+  test("UNSUPPORTED_GENERATOR: not a generator") {
+    val e = intercept[AnalysisException](
+      sql(
+        """
+          |SELECT explodedvalue.*
+          |FROM VALUES array(1, 2, 3) AS (value)
+          |LATERAL VIEW array_contains(value, 1) AS explodedvalue""".stripMargin).collect()
+    )
+
+    checkErrorClass(
+      exception = e,
+      errorClass = "UNSUPPORTED_GENERATOR",
+      errorSubClass = Some("NOT_GENERATOR"),
+      msg = "The generator is not supported: array_contains is expected to be a generator. " +

Review Comment:
   Please, quite array_contains as SQL Id: `array_contains`



-- 
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] AmplabJenkins commented on pull request #36617: [SPARK-38687][SQL] Use error classes in the compilation errors of generators

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

   Can one of the admins verify this patch?


-- 
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] panbingkun commented on pull request #36617: [SPARK-38687][SQL] Use error classes in the compilation errors of generators

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

   ping @MaxGekk 


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