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 6de527e9ee94 [SPARK-43259][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_2024 6de527e9ee94 is described below commit 6de527e9ee941bf17aa1d2b26c2a63d36e0bb946 Author: Mihailo Milosevic <mihailo.milose...@databricks.com> AuthorDate: Thu Feb 22 20:32:54 2024 +0300 [SPARK-43259][SQL] Assign a name to the error class _LEGACY_ERROR_TEMP_2024 ### What changes were proposed in this pull request? In the PR, I propose to assign the proper name `INVALID_EXPRESSION_ENCODER` to the legacy error class `_LEGACY_ERROR_TEMP_2024`, and add a test to the suite which uses `checkError()`. Also this PR improves the error message. ### Why are the changes needed? Proper name improves user experience w/ Spark SQL. ### Does this PR introduce _any_ user-facing change? Yes, the PR changes an user-facing error message. ### How was this patch tested? By running the modified test suite: ``` ./build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.errors.QueryExecutionErrorsSuite test ``` Closes #45095 from mihailom-db/SPARK-43259. Authored-by: Mihailo Milosevic <mihailo.milose...@databricks.com> Signed-off-by: Max Gekk <max.g...@gmail.com> --- .../src/main/resources/error/error-classes.json | 11 ++++++----- .../src/main/resources/error/error-states.json | 6 ++++++ docs/sql-error-conditions-sqlstates.md | 9 +++++++++ docs/sql-error-conditions.md | 6 ++++++ .../spark/sql/catalyst/encoders/package.scala | 2 +- .../spark/sql/errors/QueryExecutionErrors.scala | 12 +++++++---- .../sql/errors/QueryExecutionErrorsSuite.scala | 23 ++++++++++++++++++++-- 7 files changed, 57 insertions(+), 12 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index d4bb4920db88..17ef8e5fe469 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -1877,6 +1877,12 @@ ], "sqlState" : "F0000" }, + "INVALID_EXPRESSION_ENCODER" : { + "message" : [ + "Found an invalid expression encoder. Expects an instance of ExpressionEncoder but got <encoderType>. For more information consult '<docroot>/api/java/index.html?org/apache/spark/sql/Encoder.html'." + ], + "sqlState" : "42001" + }, "INVALID_EXTRACT_BASE_FIELD_TYPE" : { "message" : [ "Can't extract a value from <base>. Need a complex type [STRUCT, ARRAY, MAP] but got <other>." @@ -5714,11 +5720,6 @@ "Unresolved encoder expected, but <attr> was found." ] }, - "_LEGACY_ERROR_TEMP_2024" : { - "message" : [ - "Only expression encoders are supported for now." - ] - }, "_LEGACY_ERROR_TEMP_2025" : { "message" : [ "<className> must override either <m1> or <m2>." diff --git a/common/utils/src/main/resources/error/error-states.json b/common/utils/src/main/resources/error/error-states.json index e278c75ae4fa..f696866bb15d 100644 --- a/common/utils/src/main/resources/error/error-states.json +++ b/common/utils/src/main/resources/error/error-states.json @@ -2933,6 +2933,12 @@ "standard": "Y", "usedBy": ["SQL/Foundation", "PostgreSQL", "Redshift", "Oracle", "SQL Server"] }, + "42001": { + "description": "Invalid encoder error", + "origin": "Spark", + "standard": "N", + "usedBy": ["Spark"] + }, "42501": { "description": "The authorization ID does not have the privilege to perform the specified operation on the identified object.", "origin": "DB2", diff --git a/docs/sql-error-conditions-sqlstates.md b/docs/sql-error-conditions-sqlstates.md index 85f1c5c69c33..b142c7340537 100644 --- a/docs/sql-error-conditions-sqlstates.md +++ b/docs/sql-error-conditions-sqlstates.md @@ -238,6 +238,15 @@ Spark SQL uses the following `SQLSTATE` classes: <td></td> <td><a href="sql-error-conditions.html#ambiguous_reference_to_fields">AMBIGUOUS_REFERENCE_TO_FIELDS</a>, <a href="sql-error-conditions.html#invalid_column_or_field_data_type">INVALID_COLUMN_OR_FIELD_DATA_TYPE</a>, <a href="sql-error-conditions.html#invalid_extract_base_field_type">INVALID_EXTRACT_BASE_FIELD_TYPE</a>, <a href="sql-error-conditions.html#invalid_extract_field_type">INVALID_EXTRACT_FIELD_TYPE</a>, <a href="sql-error-conditions.html#invalid_field_name">INVALID_FIELD_NAME</a [...] </td> +</tr> +<tr> + <td>42001</td> + <td>Invalid encoder error</td> +</tr> +<tr> + <td></td> + <td><a href="sql-error-conditions.html#invalid_expression_encoder">INVALID_EXPRESSION_ENCODER</a> +</td> </tr> <tr> <td>42601</td> diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index e458cd5a337b..0745de995799 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -1144,6 +1144,12 @@ SQLSTATE: F0000 Executor memory `<executorMemory>` must be at least `<minSystemMemory>`. Please increase executor memory using the --executor-memory option or "`<config>`" in Spark configuration. +### INVALID_EXPRESSION_ENCODER + +[SQLSTATE: 42001](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Found an invalid expression encoder. Expects an instance of `ExpressionEncoder` but got `<encoderType>`. For more information consult '`<docroot>`/api/java/index.html?org/apache/spark/sql/Encoder.html'. + ### INVALID_EXTRACT_BASE_FIELD_TYPE [SQLSTATE: 42000](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/catalyst/encoders/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala index dfa74b6ef7e6..46e2a4098520 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/package.scala @@ -31,6 +31,6 @@ package object encoders { case e: ExpressionEncoder[A] => e.assertUnresolved() e - case _ => throw QueryExecutionErrors.unsupportedEncoderError() + case other => throw QueryExecutionErrors.invalidExpressionEncoderError(other.getClass.getName) } } 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 97ef0f9f7a13..712a276e2078 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 @@ -32,7 +32,7 @@ import org.codehaus.commons.compiler.{CompileException, InternalCompilerExceptio import org.apache.spark._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.memory.SparkOutOfMemoryError -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable} @@ -459,10 +459,14 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE messageParameters = Map("attr" -> attr.toString())) } - def unsupportedEncoderError(): SparkRuntimeException = { + def invalidExpressionEncoderError(encoderType: String): Throwable = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2024", - messageParameters = Map.empty) + errorClass = "INVALID_EXPRESSION_ENCODER", + messageParameters = Map( + "encoderType" -> encoderType, + "docroot" -> SPARK_DOC_ROOT + ) + ) } def notOverrideExpectedMethodsError( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 5f896d84ae42..fd4ebf691384 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -23,6 +23,7 @@ import java.sql.{Connection, DatabaseMetaData, Driver, DriverManager, PreparedSt import java.util.{Locale, Properties, ServiceConfigurationError} import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag import org.apache.hadoop.fs.{LocalFileSystem, Path} import org.apache.hadoop.fs.permission.FsPermission @@ -30,7 +31,7 @@ import org.mockito.Mockito.{mock, spy, when} import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, Encoder, KryoData, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{NamedParameter, UnresolvedGenerator} import org.apache.spark.sql.catalyst.expressions.{Concat, CreateArray, EmptyRow, Flatten, Grouping, Literal, RowNumber} @@ -51,7 +52,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects} import org.apache.spark.sql.streaming.StreamingQueryException import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.{ArrayType, BooleanType, DataType, DecimalType, LongType, MetadataBuilder, StructType} +import org.apache.spark.sql.types.{ArrayType, BooleanType, DataType, DecimalType, LongType, MetadataBuilder, StructField, StructType} import org.apache.spark.sql.vectorized.ColumnarArray import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.util.ThreadUtils @@ -1151,6 +1152,24 @@ class QueryExecutionErrorsSuite ) ) } + + test("SPARK-43259: Uses unsupported KryoData encoder") { + implicit val kryoEncoder = new Encoder[KryoData] { + override def schema: StructType = StructType(Array.empty[StructField]) + + override def clsTag: ClassTag[KryoData] = ClassTag(classOf[KryoData]) + } + checkError( + exception = intercept[SparkRuntimeException] { + Seq(KryoData(1), KryoData(2)).toDS() + }, + errorClass = "INVALID_EXPRESSION_ENCODER", + parameters = Map( + "encoderType" -> kryoEncoder.getClass.getName, + "docroot" -> SPARK_DOC_ROOT + ) + ) + } } class FakeFileSystemSetPermission extends LocalFileSystem { --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org