diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index 17ef8e5fe469f..5fc14d2640883 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -433,6 +433,12 @@ ], "sqlState" : "56000" }, + "CLASS_NOT_OVERRIDE_EXPECTED_METHOD" : { + "message" : [ + " must override either or ." + ], + "sqlState" : "38000" + }, "CLASS_UNSUPPORTED_BY_MAP_OBJECTS" : { "message" : [ "`MapObjects` does not support the class as resulting collection." @@ -2884,6 +2890,12 @@ ], "sqlState" : "42809" }, + "NOT_A_UNRESOLVED_ENCODER" : { + "message" : [ + "Unresolved encoder expected, but was found." + ], + "sqlState" : "42601" + }, "NOT_NULL_CONSTRAINT_VIOLATION" : { "message" : [ "Assigning a NULL is not allowed here." @@ -5715,16 +5727,6 @@ "Couldn't find a primary constructor on ." ] }, - "_LEGACY_ERROR_TEMP_2023" : { - "message" : [ - "Unresolved encoder expected, but was found." - ] - }, - "_LEGACY_ERROR_TEMP_2025" : { - "message" : [ - " must override either or ." - ] - }, "_LEGACY_ERROR_TEMP_2026" : { "message" : [ "Failed to convert value (class of ) with the type of to JSON." diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md index bb982a77fca01..faa4efb4706c5 100644 --- a/docs/sql-error-conditions.md +++ b/docs/sql-error-conditions.md @@ -366,6 +366,12 @@ Checkpoint block `` not found! Either the executor that originally checkpointed this partition is no longer alive, or the original RDD is unpersisted. If this problem persists, you may consider using `rdd.checkpoint()` instead, which is slower than local checkpointing but more fault-tolerant. +### CLASS_NOT_OVERRIDE_EXPECTED_METHOD + +[SQLSTATE: 38000](sql-error-conditions-sqlstates.html#class-38-external-routine-exception) + +`` must override either `` or ``. + ### CLASS_UNSUPPORTED_BY_MAP_OBJECTS [SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported) @@ -1663,6 +1669,12 @@ For more details see [NOT_A_CONSTANT_STRING](sql-error-conditions-not-a-constant Operation `` is not allowed for `` because it is not a partitioned table. +### NOT_A_UNRESOLVED_ENCODER + +[SQLSTATE: 42601](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) + +Unresolved encoder expected, but `` was found. + ### [NOT_NULL_CONSTRAINT_VIOLATION](sql-error-conditions-not-null-constraint-violation-error-class.html) [SQLSTATE: 42000](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation) @@ -2470,6 +2482,12 @@ The deserializer is not supported: For more details see [UNSUPPORTED_DESERIALIZER](sql-error-conditions-unsupported-deserializer-error-class.html) +### UNSUPPORTED_ENCODER + +[SQLSTATE: 0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported) + +Only expression encoders are supported for now. + ### UNSUPPORTED_EXPRESSION_GENERATED_COLUMN [SQLSTATE: 42621](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/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 654f393936368..164684454869a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -408,7 +408,7 @@ case class ExpressionEncoder[T]( * has not been done already in places where we plan to do later composition of encoders. */ def assertUnresolved(): Unit = { - (deserializer +: serializer).foreach(_.foreach { + (deserializer +: serializer).foreach(_.foreach { case a: AttributeReference if a.name != "loopVar" => throw QueryExecutionErrors.notExpectedUnresolvedEncoderError(a) case _ => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 8174328793916..a18ed6d72f8f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -565,7 +565,7 @@ abstract class UnaryExpression extends Expression with UnaryLike[Expression] { * of evaluation process, we should override [[eval]]. */ protected def nullSafeEval(input: Any): Any = - throw QueryExecutionErrors.notOverrideExpectedMethodsError("UnaryExpressions", + throw QueryExecutionErrors.notOverrideExpectedMethodsError("UnaryExpression", "eval", "nullSafeEval") /** 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 712a276e2078a..00d0b2cc2faf8 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 @@ -455,7 +455,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE def notExpectedUnresolvedEncoderError(attr: AttributeReference): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2023", + errorClass = "NOT_A_UNRESOLVED_ENCODER", messageParameters = Map("attr" -> attr.toString())) } @@ -472,7 +472,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE def notOverrideExpectedMethodsError( className: String, m1: String, m2: String): SparkRuntimeException = { new SparkRuntimeException( - errorClass = "_LEGACY_ERROR_TEMP_2025", + errorClass = "CLASS_NOT_OVERRIDE_EXPECTED_METHOD", messageParameters = Map("className" -> className, "m1" -> m1, "m2" -> m2)) } 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 fd4ebf6913844..e937d0b3490d0 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 @@ -34,9 +34,10 @@ import org.apache.spark._ 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} +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Concat, CreateArray, EmptyRow, Expression, Flatten, Grouping, Literal, RowNumber, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean import org.apache.spark.sql.catalyst.rules.RuleIdCollection import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions} @@ -53,6 +54,7 @@ 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, StructField, StructType} +import org.apache.spark.sql.types.{ArrayType, BooleanType, DataType, DecimalType, IntegerType, 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 @@ -1170,6 +1172,48 @@ class QueryExecutionErrorsSuite ) ) } + + test("ExpressionEncoder.objDeserializer should be a unsolved encoder ") { + val rowEnc = RowEncoder.encoderFor(new StructType(Array(StructField("v", IntegerType)))) + val enc: ExpressionEncoder[Row] = ExpressionEncoder(rowEnc) + val deserializer = AttributeReference.apply("v", IntegerType)() + implicit val im: ExpressionEncoder[Row] = new ExpressionEncoder[Row]( + enc.objSerializer, deserializer, enc.clsTag) + + checkError( + exception = intercept[SparkRuntimeException] { + spark.createDataset(Seq(Row(1))).collect() + }, + errorClass = "NOT_A_UNRESOLVED_ENCODER", + parameters = Map( + "attr" -> deserializer.toString + ) + ) + } + + + test("UnaryExpression should override eval or nullSafeEval") { + case class NyUnaryExpression(child: Expression) + extends UnaryExpression { + override def dataType: DataType = IntegerType + + override protected def withNewChildInternal(newChild: Expression): UnaryExpression = this + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = null + } + + checkError( + exception = intercept[SparkRuntimeException] { + NyUnaryExpression(Literal.create(1, IntegerType)).eval(EmptyRow) + }, + errorClass = "CLASS_NOT_OVERRIDE_EXPECTED_METHOD", + parameters = Map( + "className" -> "UnaryExpression", + "m1" -> "eval", + "m2" -> "nullSafeEval" + ) + ) + } } class FakeFileSystemSetPermission extends LocalFileSystem {