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

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


The following commit(s) were added to refs/heads/master by this push:
     new 69fa51cc21f [SPARK-45569][SQL] Assign name to the error 
_LEGACY_ERROR_TEMP_2152
69fa51cc21f is described below

commit 69fa51cc21f63c7ec1edfb79a535bd6024546489
Author: dengziming <dengziming1...@gmail.com>
AuthorDate: Thu Oct 19 17:23:24 2023 +0500

    [SPARK-45569][SQL] Assign name to the error _LEGACY_ERROR_TEMP_2152
    
    ### What changes were proposed in this pull request?
    Assign the name `EXPRESSION_ENCODING_FAILED` to the legacy error class 
`_LEGACY_ERROR_TEMP_2152`.
    
    ### Why are the changes needed?
    To assign proper name as a part of activity in SPARK-37935.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes, the error message will include the error class name
    
    ### How was this patch tested?
    Add a unit test to produce the error from user code.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #43396 from dengziming/SPARK-45569.
    
    Authored-by: dengziming <dengziming1...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../utils/src/main/resources/error/error-classes.json | 12 ++++++------
 docs/sql-error-conditions.md                          |  6 ++++++
 .../spark/sql/errors/QueryExecutionErrors.scala       |  3 +--
 .../catalyst/encoders/ExpressionEncoderSuite.scala    |  2 +-
 .../spark/sql/catalyst/encoders/RowEncoderSuite.scala | 12 ++++++------
 .../scala/org/apache/spark/sql/DatasetSuite.scala     | 19 +++++++++++++++++--
 6 files changed, 37 insertions(+), 17 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index dc65d7347fe..5082165a4b3 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -985,6 +985,12 @@
     ],
     "sqlState" : "42846"
   },
+  "EXPRESSION_ENCODING_FAILED" : {
+    "message" : [
+      "Failed to encode a value of the expressions: <expressions> to a row."
+    ],
+    "sqlState" : "42846"
+  },
   "EXPRESSION_TYPE_IS_NOT_ORDERABLE" : {
     "message" : [
       "Column expression <expr> cannot be sorted because its type <exprType> 
is not orderable."
@@ -5718,12 +5724,6 @@
       "Due to Scala's limited support of tuple, tuple with more than 22 
elements are not supported."
     ]
   },
-  "_LEGACY_ERROR_TEMP_2152" : {
-    "message" : [
-      "Error while encoding: <e>",
-      "<expressions>."
-    ]
-  },
   "_LEGACY_ERROR_TEMP_2154" : {
     "message" : [
       "Failed to get outer pointer for <innerCls>."
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index e555ae7476b..f22e5273746 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -574,6 +574,12 @@ For more details see 
[EXPECT_VIEW_NOT_TABLE](sql-error-conditions-expect-view-no
 
 Failed to decode a row to a value of the expressions: `<expressions>`.
 
+### EXPRESSION_ENCODING_FAILED
+
+[SQLSTATE: 
42846](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
+
+Failed to encode a value of the expressions: `<expressions>` to a row.
+
 ### EXPRESSION_TYPE_IS_NOT_ORDERABLE
 
 [SQLSTATE: 
42822](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index 957e87b4d3f..a11b929919d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -1351,9 +1351,8 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase with ExecutionE
 
   def expressionEncodingError(e: Exception, expressions: Seq[Expression]): 
SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2152",
+      errorClass = "EXPRESSION_ENCODING_FAILED",
       messageParameters = Map(
-        "e" -> e.toString(),
         "expressions" -> expressions.map(
           _.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")),
       cause = e)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
index a3ea3a462b1..8373f53446f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala
@@ -673,7 +673,7 @@ class ExpressionEncoderSuite extends 
CodegenInterpretedPlanTest with AnalysisTes
             val e = intercept[RuntimeException] {
               toRow(bigNumeric)
             }
-            assert(e.getMessage.contains("Error while encoding"))
+            assert(e.getMessage.contains("Failed to encode a value of the 
expressions:"))
             assert(e.getCause.getClass === classOf[SparkArithmeticException])
           }
         }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
index bd47537b89a..df73d50fdcd 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
@@ -276,8 +276,8 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest {
     val schema = new StructType().add("int", IntegerType)
     val encoder = ExpressionEncoder(schema)
     val e = intercept[RuntimeException](toRow(encoder, null))
-    assert(e.getMessage.contains("Null value appeared in non-nullable field"))
-    assert(e.getMessage.contains("top level Product or row object"))
+    assert(e.getCause.getMessage.contains("Null value appeared in non-nullable 
field"))
+    assert(e.getCause.getMessage.contains("top level Product or row object"))
   }
 
   test("RowEncoder should validate external type") {
@@ -286,14 +286,14 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest {
       val encoder = ExpressionEncoder(schema)
       toRow(encoder, Row(1.toShort))
     }
-    assert(e1.getMessage.contains("java.lang.Short is not a valid external 
type"))
+    assert(e1.getCause.getMessage.contains("java.lang.Short is not a valid 
external type"))
 
     val e2 = intercept[RuntimeException] {
       val schema = new StructType().add("a", StringType)
       val encoder = ExpressionEncoder(schema)
       toRow(encoder, Row(1))
     }
-    assert(e2.getMessage.contains("java.lang.Integer is not a valid external 
type"))
+    assert(e2.getCause.getMessage.contains("java.lang.Integer is not a valid 
external type"))
 
     val e3 = intercept[RuntimeException] {
       val schema = new StructType().add("a",
@@ -301,14 +301,14 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest {
       val encoder = ExpressionEncoder(schema)
       toRow(encoder, Row(1 -> "a"))
     }
-    assert(e3.getMessage.contains("scala.Tuple2 is not a valid external type"))
+    assert(e3.getCause.getMessage.contains("scala.Tuple2 is not a valid 
external type"))
 
     val e4 = intercept[RuntimeException] {
       val schema = new StructType().add("a", ArrayType(TimestampType))
       val encoder = ExpressionEncoder(schema)
       toRow(encoder, Row(Array("a")))
     }
-    assert(e4.getMessage.contains("java.lang.String is not a valid external 
type"))
+    assert(e4.getCause.getMessage.contains("java.lang.String is not a valid 
external type"))
   }
 
   private def roundTripArray[T](dt: DataType, nullable: Boolean, data: 
Array[T]): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 2579b52bf72..10a28ca24ea 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -1558,8 +1558,8 @@ class DatasetSuite extends QueryTest
 
   test("Dataset should throw RuntimeException if top-level product input 
object is null") {
     val e = intercept[RuntimeException](Seq(ClassData("a", 1), null).toDS())
-    assert(e.getMessage.contains("Null value appeared in non-nullable field"))
-    assert(e.getMessage.contains("top level Product or row object"))
+    assert(e.getCause.getMessage.contains("Null value appeared in non-nullable 
field"))
+    assert(e.getCause.getMessage.contains("top level Product or row object"))
   }
 
   test("dropDuplicates") {
@@ -2605,6 +2605,21 @@ class DatasetSuite extends QueryTest
     }
   }
 
+  test("Some(null) is unsupported when creating dataset") {
+    // Create our own encoder to avoid multiple encoders with different 
suffixes
+    implicit val enc: ExpressionEncoder[Option[String]] = ExpressionEncoder()
+    val exception = intercept[org.apache.spark.SparkRuntimeException] {
+      spark.createDataset(Seq(Some(""), None, Some(null)))
+    }
+    checkError(
+      exception = exception,
+      errorClass = "EXPRESSION_ENCODING_FAILED",
+      parameters = Map(
+        "expressions" -> enc.serializer.map(
+          _.simpleString(SQLConf.get.maxToStringFields)).mkString("\n"))
+    )
+  }
+
   test("SPARK-45386: persist with StorageLevel.NONE should give correct 
count") {
     val ds = Seq(1, 2).toDS().persist(StorageLevel.NONE)
     assert(ds.count() == 2)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to