Skip to content

Commit

Permalink
[SPARK-45569][SQL] Assign name to the error _LEGACY_ERROR_TEMP_2152
Browse files Browse the repository at this point in the history
### 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 <dengziming1993@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
  • Loading branch information
dengziming authored and MaxGekk committed Oct 19, 2023
1 parent d6d4e52 commit 69fa51c
Show file tree
Hide file tree
Showing 6 changed files with 37 additions and 17 deletions.
12 changes: 6 additions & 6 deletions common/utils/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -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."
Expand Down Expand Up @@ -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>."
Expand Down
6 changes: 6 additions & 0 deletions docs/sql-error-conditions.md
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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])
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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") {
Expand All @@ -286,29 +286,29 @@ 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",
new StructType().add("b", IntegerType).add("c", StringType))
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 = {
Expand Down
19 changes: 17 additions & 2 deletions sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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") {
Expand Down Expand Up @@ -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)
Expand Down

0 comments on commit 69fa51c

Please sign in to comment.