Skip to content

Commit

Permalink
[SPARK-43259][SQL] Assign a name to the error class _LEGACY_ERROR_TEM…
Browse files Browse the repository at this point in the history
…P_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 apache#45095 from mihailom-db/SPARK-43259.

Authored-by: Mihailo Milosevic <[email protected]>
Signed-off-by: Max Gekk <[email protected]>
  • Loading branch information
mihailom-db authored and MaxGekk committed Feb 22, 2024
1 parent 94d0d96 commit 6de527e
Show file tree
Hide file tree
Showing 7 changed files with 57 additions and 12 deletions.
11 changes: 6 additions & 5 deletions common/utils/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -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>."
Expand Down Expand Up @@ -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>."
Expand Down
6 changes: 6 additions & 0 deletions common/utils/src/main/resources/error/error-states.json
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
9 changes: 9 additions & 0 deletions docs/sql-error-conditions-sqlstates.md
Original file line number Diff line number Diff line change
Expand Up @@ -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>, <a href="sql-error-conditions.html#invalid_set_syntax">INVALID_SET_SYNTAX</a>, <a href="sql-error-conditions.html#invalid_sql_syntax">INVALID_SQL_SYNTAX</a>, <a href="sql-error-conditions.html#non_partition_column">NON_PARTITION_COLUMN</a>, <a href="not-null-constraint-violation-error-class.md">NOT_NULL_CONSTRAINT_VIOLATION</a>, <a href="sql-error-conditions.html#nullable_column_or_field">NULLABLE_COLUMN_OR_FIELD</a>, <a href="sql-error-conditions.html#nullable_row_id_attributes">NULLABLE_ROW_ID_ATTRIBUTES</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>
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 @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,14 +23,15 @@ 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
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}
Expand All @@ -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
Expand Down Expand Up @@ -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 {
Expand Down

0 comments on commit 6de527e

Please sign in to comment.