From 2cdc5b42c84d5b2cfbf9cb53c5458e52df3d6d59 Mon Sep 17 00:00:00 2001 From: Garland Zhang Date: Fri, 6 Mar 2026 09:50:14 +0000 Subject: [PATCH 1/5] Add InvalidPlanInput sql states for sql/connect --- .../resources/error/error-conditions.json | 333 ++++++++++++++++++ .../main/resources/error/error-states.json | 6 + .../common/DataTypeProtoConverter.scala | 15 +- .../common/LiteralValueProtoConverter.scala | 44 ++- .../connect/planner/InvalidInputErrors.scala | 200 ++++++++--- .../connect/planner/SparkConnectPlanner.scala | 4 +- .../sql/connect/service/SessionHolder.scala | 4 +- .../service/SparkConnectAnalyzeHandler.scala | 5 +- .../planner/InvalidInputErrorsSuite.scala | 19 +- 9 files changed, 547 insertions(+), 83 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 8f82244e1987c..e0b84d5867d73 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5612,6 +5612,339 @@ }, "sqlState" : "XXSC0" }, + "SPARK_CONNECT_INVALID_PLAN_INPUT" : { + "message" : [ + "The Spark Connect plan input is invalid." + ], + "subClass" : { + "AGGREGATE_NEEDS_PLAN_INPUT" : { + "message" : [ + "Aggregate needs a plan input" + ] + }, + "AGGREGATE_WITH_PIVOT_REQUIRES_PIVOT" : { + "message" : [ + "Aggregate with GROUP_TYPE_PIVOT requires a Pivot" + ] + }, + "ALIAS_WITH_MULTIPLE_IDENTIFIERS_AND_METADATA" : { + "message" : [ + "Alias expressions with more than 1 identifier must not use optional metadata." + ] + }, + "ARRAY_LITERAL_MISSING_DATA_TYPE" : { + "message" : [ + "Data type information is missing in the array literal." + ] + }, + "ARRAY_LITERAL_NOT_SET" : { + "message" : [ + "Array literal is not set." + ] + }, + "ASSERTION_FAILURE" : { + "message" : [ + "" + ] + }, + "CANNOT_FIND_CACHED_LOCAL_RELATION" : { + "message" : [ + "Cannot find a cached local relation for hash: " + ] + }, + "CHUNKED_CACHED_LOCAL_RELATION_WITHOUT_DATA" : { + "message" : [ + "ChunkedCachedLocalRelation should contain data." + ] + }, + "DATAFRAME_NOT_FOUND" : { + "message" : [ + "No DataFrame with id is found in the session " + ] + }, + "DATA_TYPE_UNSUPPORTED_CATALYST_TO_PROTO" : { + "message" : [ + "Does not support convert to connect proto types." + ] + }, + "DATA_TYPE_UNSUPPORTED_PROTO_TO_CATALYST" : { + "message" : [ + "Does not support convert to catalyst types." + ] + }, + "DEDUPLICATE_ALL_COLUMNS_AND_SUBSET" : { + "message" : [ + "Cannot deduplicate on both all columns and a subset of columns" + ] + }, + "DEDUPLICATE_NEEDS_INPUT" : { + "message" : [ + "Deduplicate needs a plan input" + ] + }, + "DEDUPLICATE_REQUIRES_COLUMNS_OR_ALL" : { + "message" : [ + "Deduplicate requires to either deduplicate on all columns or a subset of columns" + ] + }, + "EXCEPT_DOES_NOT_SUPPORT_UNION_BY_NAME" : { + "message" : [ + "Except does not support union_by_name" + ] + }, + "EXPECTED_NULL_VALUE" : { + "message" : [ + "Expected null value, but got " + ] + }, + "EXPECTING_SCALA_UDF" : { + "message" : [ + "Expecting a Scala UDF, but get " + ] + }, + "FIELD_CANNOT_BE_EMPTY" : { + "message" : [ + " in cannot be empty" + ] + }, + "FUNCTION_EVAL_TYPE_NOT_SUPPORTED" : { + "message" : [ + "Function with EvalType: is not supported" + ] + }, + "GROUPING_EXPRESSION_ABSENT" : { + "message" : [ + "The grouping expression cannot be absent for KeyValueGroupedDataset" + ] + }, + "INCOMPATIBLE_LITERAL_DATA_TYPE" : { + "message" : [ + "Incompatible data type for literal " + ] + }, + "INPUT_DATA_NO_SCHEMA" : { + "message" : [ + "Input data for LocalRelation does not produce a schema." + ] + }, + "INTERSECT_DOES_NOT_SUPPORT_UNION_BY_NAME" : { + "message" : [ + "Intersect does not support union_by_name" + ] + }, + "INVALID_ENUM" : { + "message" : [ + "This enum value of is invalid: ()" + ] + }, + "INVALID_JDBC_PARAMS" : { + "message" : [ + "Invalid jdbc params, please specify jdbc url and table." + ] + }, + "INVALID_ONE_OF_FIELD_NOT_SET" : { + "message" : [ + "This oneOf field in is not set: " + ] + }, + "INVALID_ONE_OF_FIELD_NOT_SUPPORTED" : { + "message" : [ + "This oneOf field message in is not supported: ()" + ] + }, + "INVALID_SQL_WITH_REFERENCES" : { + "message" : [ + " is not a valid relation for SQL with references" + ] + }, + "INVALID_WITH_RELATION_REFERENCE" : { + "message" : [ + "Invalid WithRelation reference" + ] + }, + "LAMBDA_FUNCTION_ARGUMENT_COUNT_INVALID" : { + "message" : [ + "LambdaFunction requires 1 ~ 3 arguments, but got ones!" + ] + }, + "LOCAL_RELATION_CHUNK_SIZE_LIMIT_EXCEEDED" : { + "message" : [ + "One of cached local relation chunks exceeded the limit of bytes." + ] + }, + "LOCAL_RELATION_SIZE_LIMIT_EXCEEDED" : { + "message" : [ + "Cached local relation size ( bytes) exceeds the limit ( bytes)." + ] + }, + "LOWER_BOUND_REQUIRED_IN_WINDOW_FRAME" : { + "message" : [ + "LowerBound is required in WindowFrame" + ] + }, + "MAP_LITERAL_MISSING_DATA_TYPE" : { + "message" : [ + "Data type information is missing in the map literal." + ] + }, + "MAP_LITERAL_NOT_SET" : { + "message" : [ + "Map literal is not set." + ] + }, + "MULTIPLE_PATHS_NOT_SUPPORTED_FOR_STREAMING_SOURCE" : { + "message" : [ + "Multiple paths are not supported for streaming source" + ] + }, + "NA_FILL_VALUES_EMPTY" : { + "message" : [ + "values must contains at least 1 item!" + ] + }, + "NA_FILL_VALUES_LENGTH_MISMATCH" : { + "message" : [ + "When values contains more than 1 items, values and cols should have the same length!" + ] + }, + "NOT_FOUND_CACHED_LOCAL_RELATION" : { + "message" : [ + "Not found any cached local relation with the hash: in the session with sessionUUID ." + ] + }, + "NOT_FOUND_CHUNKED_CACHED_LOCAL_RELATION" : { + "message" : [ + "Not found chunked cached local relation block with the hash: in the session with sessionUUID ." + ] + }, + "NO_HANDLER_FOR_EXTENSION" : { + "message" : [ + "No handler found for extension type: " + ] + }, + "PREDICATES_NOT_SUPPORTED_FOR_DATA_SOURCE" : { + "message" : [ + "Predicates are not supported for data sources." + ] + }, + "PYTHON_UDT_MISSING_FIELDS" : { + "message" : [ + "PythonUserDefinedType requires all the three fields: python_class, serialized_python_class and sql_type." + ] + }, + "REDUCE_SHOULD_CARRY_SCALAR_SCALA_UDF" : { + "message" : [ + "reduce should carry a scalar scala udf, but got " + ] + }, + "ROW_NOT_SUPPORTED_FOR_UDF" : { + "message" : [ + "Row is not a supported type for this UDF." + ] + }, + "SCHEMA_REQUIRED_FOR_LOCAL_RELATION" : { + "message" : [ + "Schema for LocalRelation is required when the input data is not provided." + ] + }, + "SET_OPERATION_MUST_HAVE_TWO_INPUTS" : { + "message" : [ + "Set operation must have 2 inputs" + ] + }, + "SQL_COMMAND_EXPECTS_SQL_OR_WITH_RELATIONS" : { + "message" : [ + "SQL command expects either a SQL or a WithRelations, but got " + ] + }, + "STREAMING_QUERY_NOT_FOUND" : { + "message" : [ + "Streaming query is not found" + ] + }, + "STREAMING_QUERY_RUN_ID_MISMATCH" : { + "message" : [ + "Run id mismatch for query id . Run id in the request does not match one on the server . The query might have restarted." + ] + }, + "STRUCT_LITERAL_MISSING_DATA_TYPE" : { + "message" : [ + "Data type information is missing in the struct literal." + ] + }, + "STRUCT_LITERAL_NOT_SET" : { + "message" : [ + "Struct literal is not set." + ] + }, + "UDT_TYPE_FIELD_INVALID" : { + "message" : [ + "UserDefinedType requires the 'type' field to be 'udt', but got ''." + ] + }, + "UNION_BY_NAME_ALLOW_MISSING_COL_REQUIRES_BY_NAME" : { + "message" : [ + "UnionByName `allowMissingCol` can be true only if `byName` is true." + ] + }, + "UNKNOWN_ANALYZE_METHOD" : { + "message" : [ + "Unknown Analyze Method !" + ] + }, + "UNRESOLVED_NAMED_LAMBDA_VARIABLE_REQUIRES_NAME_PART" : { + "message" : [ + "UnresolvedNamedLambdaVariable requires at least one name part!" + ] + }, + "UNRESOLVED_STAR_TARGET_INVALID" : { + "message" : [ + "UnresolvedStar requires a unparsed target ending with '.*', but got ." + ] + }, + "UNRESOLVED_STAR_WITH_BOTH_TARGET_AND_PLAN_ID" : { + "message" : [ + "UnresolvedStar with both target and plan id is not supported." + ] + }, + "UNSUPPORTED_LITERAL_TYPE" : { + "message" : [ + "Unsupported Literal Type: " + ] + }, + "UNSUPPORTED_LITERAL_TYPE_WITH_NUMBER" : { + "message" : [ + "Unsupported Literal Type: ()" + ] + }, + "UNSUPPORTED_USER_DEFINED_FUNCTION_IMPLEMENTATION" : { + "message" : [ + "Unsupported UserDefinedFunction implementation: " + ] + }, + "UPPER_BOUND_REQUIRED_IN_WINDOW_FRAME" : { + "message" : [ + "UpperBound is required in WindowFrame" + ] + }, + "USING_COLUMNS_OR_JOIN_CONDITION_SET_IN_JOIN" : { + "message" : [ + "Using columns or join conditions cannot be set at the same time in Join" + ] + }, + "WINDOW_FUNCTION_REQUIRED" : { + "message" : [ + "WindowFunction is required in WindowExpression" + ] + }, + "WITH_COLUMNS_REQUIRE_SINGLE_NAME_PART" : { + "message" : [ + "WithColumns require column name only contains one name part, but got " + ] + } + }, + "sqlState" : "XXSC1" + }, "SPARK_JOB_CANCELLED" : { "message" : [ "Job cancelled " diff --git a/common/utils/src/main/resources/error/error-states.json b/common/utils/src/main/resources/error/error-states.json index c2b2bb2ed4638..391a839e7092f 100644 --- a/common/utils/src/main/resources/error/error-states.json +++ b/common/utils/src/main/resources/error/error-states.json @@ -7536,6 +7536,12 @@ "standard": "N", "usedBy": ["Spark"] }, + "XXSC1": { + "description": "Connect Server - Invalid Plan Input", + "origin": "Spark", + "standard": "N", + "usedBy": ["Spark"] + }, "XXKD0": { "description": "Analysis - Bad plan", "origin": "Databricks", diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala index 0500ca478dada..a1406684b85a4 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala @@ -89,7 +89,9 @@ object DataTypeProtoConverter { case proto.DataType.KindCase.UDT => toCatalystUDT(t.getUdt) case _ => - throw InvalidPlanInput(s"Does not support convert ${t.getKindCase} to catalyst types.") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.DATA_TYPE_UNSUPPORTED_PROTO_TO_CATALYST", + Map("kindCase" -> t.getKindCase.toString)) } } @@ -147,7 +149,8 @@ object DataTypeProtoConverter { private def toCatalystUDT(t: proto.DataType.UDT): UserDefinedType[_] = { if (t.getType != "udt") { throw InvalidPlanInput( - s"""UserDefinedType requires the 'type' field to be 'udt', but got '${t.getType}'.""") + "SPARK_CONNECT_INVALID_PLAN_INPUT.UDT_TYPE_FIELD_INVALID", + Map("udtType" -> t.getType)) } if (t.hasJvmClass) { @@ -158,8 +161,8 @@ object DataTypeProtoConverter { } else { if (!t.hasPythonClass || !t.hasSerializedPythonClass || !t.hasSqlType) { throw InvalidPlanInput( - "PythonUserDefinedType requires all the three fields: " + - "python_class, serialized_python_class and sql_type.") + "SPARK_CONNECT_INVALID_PLAN_INPUT.PYTHON_UDT_MISSING_FIELDS", + Map.empty) } new PythonUserDefinedType( @@ -389,7 +392,9 @@ object DataTypeProtoConverter { } case _ => - throw InvalidPlanInput(s"Does not support convert ${t.typeName} to connect proto types.") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.DATA_TYPE_UNSUPPORTED_CATALYST_TO_PROTO", + Map("typeName" -> t.typeName)) } } } diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala index 63c43f956d78b..2836b7f1341bf 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala @@ -387,7 +387,9 @@ object LiteralValueProtoConverter { private def getScalaConverter(dataType: proto.DataType): proto.Expression.Literal => Any = { val converter: proto.Expression.Literal => Any = dataType.getKindCase match { case proto.DataType.KindCase.NULL => - v => throw InvalidPlanInput(s"Expected null value, but got ${v.getLiteralTypeCase}") + v => throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.EXPECTED_NULL_VALUE", + Map("literalTypeCase" -> v.getLiteralTypeCase.toString)) case proto.DataType.KindCase.SHORT => v => v.getShort.toShort case proto.DataType.KindCase.INTEGER => v => v.getInteger case proto.DataType.KindCase.LONG => v => v.getLong @@ -421,7 +423,9 @@ object LiteralValueProtoConverter { case proto.DataType.KindCase.STRUCT => v => toScalaStructInternal(v, dataType.getStruct) case _ => - throw InvalidPlanInput(s"Unsupported Literal Type: ${dataType.getKindCase}") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNSUPPORTED_LITERAL_TYPE", + Map("kindCase" -> dataType.getKindCase.toString)) } v => if (v.hasNull) null else converter(v) } @@ -558,7 +562,9 @@ object LiteralValueProtoConverter { .setContainsNull(true) .build()) } else { - throw InvalidPlanInput("Data type information is missing in the array literal.") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.ARRAY_LITERAL_MISSING_DATA_TYPE", + Map.empty) } case proto.Expression.Literal.LiteralTypeCase.MAP => if (literal.getMap.hasKeyType && literal.getMap.hasValueType) { @@ -570,18 +576,24 @@ object LiteralValueProtoConverter { .setValueContainsNull(true) .build()) } else { - throw InvalidPlanInput("Data type information is missing in the map literal.") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.MAP_LITERAL_MISSING_DATA_TYPE", + Map.empty) } case proto.Expression.Literal.LiteralTypeCase.STRUCT => if (literal.getStruct.hasStructType) { builder.setStruct(literal.getStruct.getStructType.getStruct) } else { - throw InvalidPlanInput("Data type information is missing in the struct literal.") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.STRUCT_LITERAL_MISSING_DATA_TYPE", + Map.empty) } case _ => throw InvalidPlanInput( - s"Unsupported Literal Type: ${literal.getLiteralTypeCase.name}" + - s"(${literal.getLiteralTypeCase.getNumber})") + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNSUPPORTED_LITERAL_TYPE_WITH_NUMBER", + Map( + "name" -> literal.getLiteralTypeCase.name, + "number" -> literal.getLiteralTypeCase.getNumber.toString)) } builder.build() } @@ -589,8 +601,10 @@ object LiteralValueProtoConverter { if (!isCompatible(literal.getLiteralTypeCase, dataType.getKindCase)) { throw InvalidPlanInput( - s"Incompatible data type ${dataType.getKindCase} " + - s"for literal ${literal.getLiteralTypeCase}") + "SPARK_CONNECT_INVALID_PLAN_INPUT.INCOMPATIBLE_LITERAL_DATA_TYPE", + Map( + "dataTypeKindCase" -> dataType.getKindCase.toString, + "literalTypeCase" -> literal.getLiteralTypeCase.toString)) } dataType @@ -600,7 +614,9 @@ object LiteralValueProtoConverter { literal: proto.Expression.Literal, arrayType: proto.DataType.Array): Array[_] = { if (!literal.hasArray) { - throw InvalidPlanInput("Array literal is not set.") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.ARRAY_LITERAL_NOT_SET", + Map.empty) } val array = literal.getArray def makeArrayData[T](converter: proto.Expression.Literal => T)(implicit @@ -620,7 +636,9 @@ object LiteralValueProtoConverter { literal: proto.Expression.Literal, mapType: proto.DataType.Map): mutable.Map[_, _] = { if (!literal.hasMap) { - throw InvalidPlanInput("Map literal is not set.") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.MAP_LITERAL_NOT_SET", + Map.empty) } val map = literal.getMap def makeMapData[K, V]( @@ -646,7 +664,9 @@ object LiteralValueProtoConverter { literal: proto.Expression.Literal, structType: proto.DataType.Struct): Any = { if (!literal.hasStruct) { - throw InvalidPlanInput("Struct literal is not set.") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.STRUCT_LITERAL_NOT_SET", + Map.empty) } val struct = literal.getStruct val structData = Array.tabulate(struct.getElementsCount) { i => diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala index f4a6913d1eab9..4fcfe6cdaf97a 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala @@ -30,29 +30,40 @@ import org.apache.spark.sql.types.DataType object InvalidInputErrors { - def noHandlerFoundForExtension(extensionTypeUrl: String): InvalidPlanInput = { - InvalidPlanInput(s"No handler found for extension type: $extensionTypeUrl") - } + def noHandlerFoundForExtension(extensionTypeUrl: String): InvalidPlanInput = + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.NO_HANDLER_FOR_EXTENSION", + Map("extensionTypeUrl" -> extensionTypeUrl)) def invalidSQLWithReferences(query: proto.WithRelations): InvalidPlanInput = - InvalidPlanInput(s"$query is not a valid relation for SQL with references") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.INVALID_SQL_WITH_REFERENCES", + Map("query" -> query.toString)) def naFillValuesEmpty(): InvalidPlanInput = - InvalidPlanInput("values must contains at least 1 item!") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.NA_FILL_VALUES_EMPTY", + Map.empty) def naFillValuesLengthMismatch(): InvalidPlanInput = InvalidPlanInput( - "When values contains more than 1 items, values and cols should have the same length!") + "SPARK_CONNECT_INVALID_PLAN_INPUT.NA_FILL_VALUES_LENGTH_MISMATCH", + Map.empty) def deduplicateNeedsInput(): InvalidPlanInput = - InvalidPlanInput("Deduplicate needs a plan input") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.DEDUPLICATE_NEEDS_INPUT", + Map.empty) def deduplicateAllColumnsAndSubset(): InvalidPlanInput = - InvalidPlanInput("Cannot deduplicate on both all columns and a subset of columns") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.DEDUPLICATE_ALL_COLUMNS_AND_SUBSET", + Map.empty) def deduplicateRequiresColumnsOrAll(): InvalidPlanInput = InvalidPlanInput( - "Deduplicate requires to either deduplicate on all columns or a subset of columns") + "SPARK_CONNECT_INVALID_PLAN_INPUT.DEDUPLICATE_REQUIRES_COLUMNS_OR_ALL", + Map.empty) def invalidDeduplicateColumn(colName: String, fieldNames: String): InvalidPlanInput = InvalidPlanInput( @@ -60,47 +71,66 @@ object InvalidInputErrors { Map("colName" -> colName, "fieldNames" -> fieldNames)) def functionEvalTypeNotSupported(evalType: Int): InvalidPlanInput = - InvalidPlanInput(s"Function with EvalType: $evalType is not supported") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.FUNCTION_EVAL_TYPE_NOT_SUPPORTED", + Map("evalType" -> evalType.toString)) def groupingExpressionAbsentForKeyValueGroupedDataset(): InvalidPlanInput = - InvalidPlanInput("The grouping expression cannot be absent for KeyValueGroupedDataset") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.GROUPING_EXPRESSION_ABSENT", + Map.empty) def expectingScalaUdfButGot(exprType: proto.Expression.ExprTypeCase): InvalidPlanInput = - InvalidPlanInput(s"Expecting a Scala UDF, but get $exprType") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.EXPECTING_SCALA_UDF", + Map("exprType" -> exprType.toString)) def rowNotSupportedForUdf(errorType: String): InvalidPlanInput = - InvalidPlanInput(s"Row is not a supported $errorType type for this UDF.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.ROW_NOT_SUPPORTED_FOR_UDF", + Map("errorType" -> errorType)) def notFoundCachedLocalRelation(hash: String, sessionUUID: String): InvalidPlanInput = InvalidPlanInput( - s"Not found any cached local relation with the hash: " + - s"$hash in the session with sessionUUID $sessionUUID.") + "SPARK_CONNECT_INVALID_PLAN_INPUT.NOT_FOUND_CACHED_LOCAL_RELATION", + Map("hash" -> hash, "sessionUUID" -> sessionUUID)) def notFoundChunkedCachedLocalRelationBlock( hash: String, sessionUUID: String): InvalidPlanInput = InvalidPlanInput( - s"Not found chunked cached local relation block with the hash: " + - s"$hash in the session with sessionUUID $sessionUUID.") + "SPARK_CONNECT_INVALID_PLAN_INPUT.NOT_FOUND_CHUNKED_CACHED_LOCAL_RELATION", + Map("hash" -> hash, "sessionUUID" -> sessionUUID)) def localRelationSizeLimitExceeded(actualSize: Long, limit: Long): InvalidPlanInput = InvalidPlanInput( - s"Cached local relation size ($actualSize bytes) exceeds the limit ($limit bytes).") + "SPARK_CONNECT_INVALID_PLAN_INPUT.LOCAL_RELATION_SIZE_LIMIT_EXCEEDED", + Map("actualSize" -> actualSize.toString, "limit" -> limit.toString)) def localRelationChunkSizeLimitExceeded(limit: Long): InvalidPlanInput = - InvalidPlanInput(s"One of cached local relation chunks exceeded the limit of $limit bytes.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.LOCAL_RELATION_CHUNK_SIZE_LIMIT_EXCEEDED", + Map("limit" -> limit.toString)) def withColumnsRequireSingleNamePart(got: String): InvalidPlanInput = - InvalidPlanInput(s"WithColumns require column name only contains one name part, but got $got") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.WITH_COLUMNS_REQUIRE_SINGLE_NAME_PART", + Map("got" -> got)) def inputDataForLocalRelationNoSchema(): InvalidPlanInput = - InvalidPlanInput("Input data for LocalRelation does not produce a schema.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.INPUT_DATA_NO_SCHEMA", + Map.empty) def chunkedCachedLocalRelationWithoutData(): InvalidPlanInput = - InvalidPlanInput("ChunkedCachedLocalRelation should contain data.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.CHUNKED_CACHED_LOCAL_RELATION_WITHOUT_DATA", + Map.empty) def schemaRequiredForLocalRelation(): InvalidPlanInput = - InvalidPlanInput("Schema for LocalRelation is required when the input data is not provided.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.SCHEMA_REQUIRED_FOR_LOCAL_RELATION", + Map.empty) def invalidSchemaStringNonStructType(schema: String, dataType: DataType): InvalidPlanInput = InvalidPlanInput( @@ -108,18 +138,27 @@ object InvalidInputErrors { Map("inputSchema" -> quoteByDefault(schema), "dataType" -> toSQLType(dataType))) def invalidJdbcParams(): InvalidPlanInput = - InvalidPlanInput("Invalid jdbc params, please specify jdbc url and table.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.INVALID_JDBC_PARAMS", + Map.empty) def predicatesNotSupportedForDataSource(format: String): InvalidPlanInput = - InvalidPlanInput(s"Predicates are not supported for $format data sources.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.PREDICATES_NOT_SUPPORTED_FOR_DATA_SOURCE", + Map("format" -> format)) def multiplePathsNotSupportedForStreamingSource(): InvalidPlanInput = - InvalidPlanInput("Multiple paths are not supported for streaming source") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.MULTIPLE_PATHS_NOT_SUPPORTED_FOR_STREAMING_SOURCE", + Map.empty) def invalidEnum(protoEnum: Enum[_] with ProtocolMessageEnum): InvalidPlanInput = InvalidPlanInput( - s"This enum value of ${protoEnum.getDescriptorForType.getFullName}" + - s" is invalid: ${protoEnum.name()}(${protoEnum.getNumber})") + "SPARK_CONNECT_INVALID_PLAN_INPUT.INVALID_ENUM", + Map( + "fullName" -> protoEnum.getDescriptorForType.getFullName, + "name" -> protoEnum.name(), + "number" -> protoEnum.getNumber.toString)) def invalidOneOfField( enumCase: Enum[_] with EnumLite, @@ -127,90 +166,141 @@ object InvalidInputErrors { // If the oneOf field is not set, the enum number will be 0. if (enumCase.getNumber == 0) { InvalidPlanInput( - s"This oneOf field in ${descriptor.getFullName} is not set: ${enumCase.name()}") + "SPARK_CONNECT_INVALID_PLAN_INPUT.INVALID_ONE_OF_FIELD_NOT_SET", + Map("fullName" -> descriptor.getFullName, "name" -> enumCase.name())) } else { InvalidPlanInput( - s"This oneOf field message in ${descriptor.getFullName} is not supported: " + - s"${enumCase.name()}(${enumCase.getNumber})") + "SPARK_CONNECT_INVALID_PLAN_INPUT.INVALID_ONE_OF_FIELD_NOT_SUPPORTED", + Map( + "fullName" -> descriptor.getFullName, + "name" -> enumCase.name(), + "number" -> enumCase.getNumber.toString)) } } def cannotBeEmpty(fieldName: String, descriptor: Descriptor): InvalidPlanInput = - InvalidPlanInput(s"$fieldName in ${descriptor.getFullName} cannot be empty") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.FIELD_CANNOT_BE_EMPTY", + Map("fieldName" -> fieldName, "fullName" -> descriptor.getFullName)) def invalidSchemaTypeNonStruct(dataType: DataType): InvalidPlanInput = InvalidPlanInput("INVALID_SCHEMA_TYPE_NON_STRUCT", Map("dataType" -> toSQLType(dataType))) def lambdaFunctionArgumentCountInvalid(got: Int): InvalidPlanInput = - InvalidPlanInput(s"LambdaFunction requires 1 ~ 3 arguments, but got $got ones!") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.LAMBDA_FUNCTION_ARGUMENT_COUNT_INVALID", + Map("got" -> got.toString)) def aliasWithMultipleIdentifiersAndMetadata(): InvalidPlanInput = InvalidPlanInput( - "Alias expressions with more than 1 identifier must not use optional metadata.") + "SPARK_CONNECT_INVALID_PLAN_INPUT.ALIAS_WITH_MULTIPLE_IDENTIFIERS_AND_METADATA", + Map.empty) def unresolvedStarTargetInvalid(target: String): InvalidPlanInput = InvalidPlanInput( - s"UnresolvedStar requires a unparsed target ending with '.*', but got $target.") + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNRESOLVED_STAR_TARGET_INVALID", + Map("target" -> target)) def unresolvedStarWithBothTargetAndPlanId(): InvalidPlanInput = - InvalidPlanInput("UnresolvedStar with both target and plan id is not supported.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNRESOLVED_STAR_WITH_BOTH_TARGET_AND_PLAN_ID", + Map.empty) def windowFunctionRequired(): InvalidPlanInput = - InvalidPlanInput("WindowFunction is required in WindowExpression") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.WINDOW_FUNCTION_REQUIRED", + Map.empty) def lowerBoundRequiredInWindowFrame(): InvalidPlanInput = - InvalidPlanInput("LowerBound is required in WindowFrame") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.LOWER_BOUND_REQUIRED_IN_WINDOW_FRAME", + Map.empty) def upperBoundRequiredInWindowFrame(): InvalidPlanInput = - InvalidPlanInput("UpperBound is required in WindowFrame") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.UPPER_BOUND_REQUIRED_IN_WINDOW_FRAME", + Map.empty) def setOperationMustHaveTwoInputs(): InvalidPlanInput = - InvalidPlanInput("Set operation must have 2 inputs") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.SET_OPERATION_MUST_HAVE_TWO_INPUTS", + Map.empty) def exceptDoesNotSupportUnionByName(): InvalidPlanInput = - InvalidPlanInput("Except does not support union_by_name") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.EXCEPT_DOES_NOT_SUPPORT_UNION_BY_NAME", + Map.empty) def intersectDoesNotSupportUnionByName(): InvalidPlanInput = - InvalidPlanInput("Intersect does not support union_by_name") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.INTERSECT_DOES_NOT_SUPPORT_UNION_BY_NAME", + Map.empty) def aggregateNeedsPlanInput(): InvalidPlanInput = - InvalidPlanInput("Aggregate needs a plan input") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.AGGREGATE_NEEDS_PLAN_INPUT", + Map.empty) def aggregateWithPivotRequiresPivot(): InvalidPlanInput = - InvalidPlanInput("Aggregate with GROUP_TYPE_PIVOT requires a Pivot") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.AGGREGATE_WITH_PIVOT_REQUIRES_PIVOT", + Map.empty) def invalidWithRelationReference(): InvalidPlanInput = - InvalidPlanInput("Invalid WithRelation reference") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.INVALID_WITH_RELATION_REFERENCE", + Map.empty) def assertionFailure(message: String): InvalidPlanInput = - InvalidPlanInput(message) + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.ASSERTION_FAILURE", + Map("message" -> message)) def unresolvedNamedLambdaVariableRequiresNamePart(): InvalidPlanInput = - InvalidPlanInput("UnresolvedNamedLambdaVariable requires at least one name part!") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNRESOLVED_NAMED_LAMBDA_VARIABLE_REQUIRES_NAME_PART", + Map.empty) def usingColumnsOrJoinConditionSetInJoin(): InvalidPlanInput = - InvalidPlanInput("Using columns or join conditions cannot be set at the same time in Join") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.USING_COLUMNS_OR_JOIN_CONDITION_SET_IN_JOIN", + Map.empty) def sqlCommandExpectsSqlOrWithRelations(other: proto.Relation.RelTypeCase): InvalidPlanInput = - InvalidPlanInput(s"SQL command expects either a SQL or a WithRelations, but got $other") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.SQL_COMMAND_EXPECTS_SQL_OR_WITH_RELATIONS", + Map("other" -> other.toString)) def reduceShouldCarryScalarScalaUdf(got: mutable.Buffer[proto.Expression]): InvalidPlanInput = - InvalidPlanInput(s"reduce should carry a scalar scala udf, but got $got") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.REDUCE_SHOULD_CARRY_SCALAR_SCALA_UDF", + Map("got" -> got.toString)) def unionByNameAllowMissingColRequiresByName(): InvalidPlanInput = - InvalidPlanInput("UnionByName `allowMissingCol` can be true only if `byName` is true.") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNION_BY_NAME_ALLOW_MISSING_COL_REQUIRES_BY_NAME", + Map.empty) def unsupportedUserDefinedFunctionImplementation(clazz: Class[_]): InvalidPlanInput = - InvalidPlanInput(s"Unsupported UserDefinedFunction implementation: ${clazz}") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNSUPPORTED_USER_DEFINED_FUNCTION_IMPLEMENTATION", + Map("clazz" -> clazz.toString)) def streamingQueryRunIdMismatch( id: String, runId: String, serverRunId: String): InvalidPlanInput = InvalidPlanInput( - s"Run id mismatch for query id $id. Run id in the request $runId " + - s"does not match one on the server $serverRunId. The query might have restarted.") + "SPARK_CONNECT_INVALID_PLAN_INPUT.STREAMING_QUERY_RUN_ID_MISMATCH", + Map("id" -> id, "runId" -> runId, "serverRunId" -> serverRunId)) def streamingQueryNotFound(id: String): InvalidPlanInput = - InvalidPlanInput(s"Streaming query $id is not found") + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.STREAMING_QUERY_NOT_FOUND", + Map("id" -> id)) + + def cannotFindCachedLocalRelation(hash: String): InvalidPlanInput = + InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.CANNOT_FIND_CACHED_LOCAL_RELATION", + Map("hash" -> hash)) } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index ee8180d5e6f8b..5da4b022942be 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -59,7 +59,7 @@ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.classic.{Catalog, DataFrameWriter, Dataset, MergeIntoWriter, RelationalGroupedDataset, SparkSession, TypedAggUtils, UserDefinedFunctionUtils} import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.connect.client.arrow.ArrowSerializer -import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, InvalidPlanInput, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE import org.apache.spark.sql.connect.ml.MLHandler import org.apache.spark.sql.connect.pipelines.PipelinesHandler @@ -1332,7 +1332,7 @@ class SparkConnectPlanner( private def transformCachedLocalRelation(rel: proto.CachedLocalRelation): LogicalPlan = { val blockManager = session.sparkContext.env.blockManager val blockId = session.artifactManager.getCachedBlockId(rel.getHash).getOrElse { - throw InvalidPlanInput(s"Cannot find a cached local relation for hash: ${rel.getHash}") + throw InvalidInputErrors.cannotFindCachedLocalRelation(rel.getHash) } val bytes = blockManager.getLocalBytes(blockId) bytes diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index e5f9b7fe85f1d..49af899056b54 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -465,7 +465,9 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio private[connect] def getDataFrameOrThrow(dfId: String): DataFrame = { Option(dataFrameCache.get(dfId)) .getOrElse { - throw InvalidPlanInput(s"No DataFrame with id $dfId is found in the session $sessionId") + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.DATAFRAME_NOT_FOUND", + Map("dfId" -> dfId, "sessionId" -> sessionId)) } } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala index 72029cafaa634..8450e3f4201c5 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala @@ -244,7 +244,10 @@ private[connect] class SparkConnectAnalyzeHandler( // RequestDecompressionInterceptor.decompressAnalyzePlanRequest() to handle // this case. The interceptor has a default case that throws UnsupportedOperationException // for unhandled cases, which will fail tests and block CI if you forget to update it. - case other => throw InvalidPlanInput(s"Unknown Analyze Method $other!") + case other => + throw InvalidPlanInput( + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNKNOWN_ANALYZE_METHOD", + Map("other" -> other.toString)) } builder diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala index 07c377a77df5f..0633488b52234 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala @@ -97,8 +97,8 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { }), TestCase( name = "Deduplicate needs input", - expectedErrorCondition = "INTERNAL_ERROR", - expectedParameters = Map("message" -> "Deduplicate needs a plan input"), + expectedErrorCondition = "SPARK_CONNECT_INVALID_PLAN_INPUT.DEDUPLICATE_NEEDS_INPUT", + expectedParameters = Map.empty, invalidInput = { val deduplicate = proto.Deduplicate .newBuilder() @@ -109,9 +109,10 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { }), TestCase( name = "Catalog not set", - expectedErrorCondition = "INTERNAL_ERROR", + expectedErrorCondition = + "SPARK_CONNECT_INVALID_PLAN_INPUT.INVALID_ONE_OF_FIELD_NOT_SET", expectedParameters = - Map("message" -> "This oneOf field in spark.connect.Catalog is not set: CATTYPE_NOT_SET"), + Map("fullName" -> "spark.connect.Catalog", "name" -> "CATTYPE_NOT_SET"), invalidInput = { val catalog = proto.Catalog .newBuilder() @@ -126,12 +127,16 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { // Run all test cases testCases.foreach { testCase => test(s"${testCase.name}") { + val exception = intercept[InvalidPlanInput] { + transform(testCase.invalidInput) + } checkError( - exception = intercept[InvalidPlanInput] { - transform(testCase.invalidInput) - }, + exception = exception, condition = testCase.expectedErrorCondition, parameters = testCase.expectedParameters) + if (testCase.expectedErrorCondition.startsWith("SPARK_CONNECT_INVALID_PLAN_INPUT")) { + assert(exception.getSqlState == "XXSC1") + } } } From 969da67e823834c7bc958ee1f79526e9711efadc Mon Sep 17 00:00:00 2001 From: Garland Zhang Date: Fri, 6 Mar 2026 10:03:41 +0000 Subject: [PATCH 2/5] update --- .../resources/error/error-conditions.json | 7 +--- .../common/LiteralValueProtoConverter.scala | 8 ++-- .../planner/InvalidInputErrorsSuite.scala | 41 ++++++++++++++++++- 3 files changed, 44 insertions(+), 12 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index e0b84d5867d73..0f291a2648e82 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5909,12 +5909,7 @@ }, "UNSUPPORTED_LITERAL_TYPE" : { "message" : [ - "Unsupported Literal Type: " - ] - }, - "UNSUPPORTED_LITERAL_TYPE_WITH_NUMBER" : { - "message" : [ - "Unsupported Literal Type: ()" + "Unsupported Literal Type: " ] }, "UNSUPPORTED_USER_DEFINED_FUNCTION_IMPLEMENTATION" : { diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala index 2836b7f1341bf..0678290a2af41 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala @@ -425,7 +425,7 @@ object LiteralValueProtoConverter { case _ => throw InvalidPlanInput( "SPARK_CONNECT_INVALID_PLAN_INPUT.UNSUPPORTED_LITERAL_TYPE", - Map("kindCase" -> dataType.getKindCase.toString)) + Map("typeInfo" -> dataType.getKindCase.toString)) } v => if (v.hasNull) null else converter(v) } @@ -590,10 +590,8 @@ object LiteralValueProtoConverter { } case _ => throw InvalidPlanInput( - "SPARK_CONNECT_INVALID_PLAN_INPUT.UNSUPPORTED_LITERAL_TYPE_WITH_NUMBER", - Map( - "name" -> literal.getLiteralTypeCase.name, - "number" -> literal.getLiteralTypeCase.getNumber.toString)) + "SPARK_CONNECT_INVALID_PLAN_INPUT.UNSUPPORTED_LITERAL_TYPE", + Map("typeInfo" -> s"${literal.getLiteralTypeCase.name}(${literal.getLiteralTypeCase.getNumber})")) } builder.build() } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala index 0633488b52234..4f75ed163f9c6 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.connect.proto import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, InvalidPlanInput} -import org.apache.spark.sql.connect.planner.SparkConnectPlanTest +import org.apache.spark.sql.connect.planner.{InvalidInputErrors, SparkConnectPlanTest} import org.apache.spark.sql.types._ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { @@ -140,6 +140,45 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { } } + test("noHandlerFoundForExtension") { + val ex = InvalidInputErrors.noHandlerFoundForExtension("foo.bar.Ext") + assert(ex.getCondition.contains("NO_HANDLER_FOR_EXTENSION")) + assert(ex.getMessage.contains("foo.bar.Ext")) + assert(ex.getSqlState == "XXSC1") + } + + test("notFoundCachedLocalRelation") { + val ex = InvalidInputErrors.notFoundCachedLocalRelation("abc123", "sess-uuid") + assert(ex.getCondition.contains("NOT_FOUND_CACHED_LOCAL_RELATION")) + assert(ex.getMessage.contains("abc123")) + assert(ex.getMessage.contains("sess-uuid")) + assert(ex.getSqlState == "XXSC1") + } + + test("localRelationSizeLimitExceeded") { + val ex = InvalidInputErrors.localRelationSizeLimitExceeded(1000L, 500L) + assert(ex.getCondition.contains("LOCAL_RELATION_SIZE_LIMIT_EXCEEDED")) + assert(ex.getMessage.contains("1000")) + assert(ex.getMessage.contains("500")) + assert(ex.getSqlState == "XXSC1") + } + + test("functionEvalTypeNotSupported") { + val ex = InvalidInputErrors.functionEvalTypeNotSupported(42) + assert(ex.getCondition.contains("FUNCTION_EVAL_TYPE_NOT_SUPPORTED")) + assert(ex.getMessage.contains("42")) + assert(ex.getSqlState == "XXSC1") + } + + test("streamingQueryRunIdMismatch") { + val ex = InvalidInputErrors.streamingQueryRunIdMismatch("q1", "run1", "run2") + assert(ex.getCondition.contains("STREAMING_QUERY_RUN_ID_MISMATCH")) + assert(ex.getMessage.contains("q1")) + assert(ex.getMessage.contains("run1")) + assert(ex.getMessage.contains("run2")) + assert(ex.getSqlState == "XXSC1") + } + // Helper case class to define test cases case class TestCase( name: String, From a7b1edfe5ebf4303c6f33656f66627046e1ffaf8 Mon Sep 17 00:00:00 2001 From: Garland Zhang Date: Fri, 6 Mar 2026 10:17:11 +0000 Subject: [PATCH 3/5] update --- .../src/main/resources/error/error-conditions.json | 2 +- .../utils/src/main/resources/error/error-states.json | 6 ------ .../connect/common/LiteralValueProtoConverter.scala | 3 ++- .../connect/planner/InvalidInputErrorsSuite.scala | 12 ++++++------ 4 files changed, 9 insertions(+), 14 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 0f291a2648e82..12b697ea72eac 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5938,7 +5938,7 @@ ] } }, - "sqlState" : "XXSC1" + "sqlState" : "42000" }, "SPARK_JOB_CANCELLED" : { "message" : [ diff --git a/common/utils/src/main/resources/error/error-states.json b/common/utils/src/main/resources/error/error-states.json index 391a839e7092f..c2b2bb2ed4638 100644 --- a/common/utils/src/main/resources/error/error-states.json +++ b/common/utils/src/main/resources/error/error-states.json @@ -7536,12 +7536,6 @@ "standard": "N", "usedBy": ["Spark"] }, - "XXSC1": { - "description": "Connect Server - Invalid Plan Input", - "origin": "Spark", - "standard": "N", - "usedBy": ["Spark"] - }, "XXKD0": { "description": "Analysis - Bad plan", "origin": "Databricks", diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala index 0678290a2af41..cd17085888366 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala @@ -589,9 +589,10 @@ object LiteralValueProtoConverter { Map.empty) } case _ => + val literalCase = literal.getLiteralTypeCase throw InvalidPlanInput( "SPARK_CONNECT_INVALID_PLAN_INPUT.UNSUPPORTED_LITERAL_TYPE", - Map("typeInfo" -> s"${literal.getLiteralTypeCase.name}(${literal.getLiteralTypeCase.getNumber})")) + Map("typeInfo" -> s"${literalCase.name}(${literalCase.getNumber})")) } builder.build() } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala index 4f75ed163f9c6..811ede68a2294 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/InvalidInputErrorsSuite.scala @@ -135,7 +135,7 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { condition = testCase.expectedErrorCondition, parameters = testCase.expectedParameters) if (testCase.expectedErrorCondition.startsWith("SPARK_CONNECT_INVALID_PLAN_INPUT")) { - assert(exception.getSqlState == "XXSC1") + assert(exception.getSqlState == "42000") } } } @@ -144,7 +144,7 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { val ex = InvalidInputErrors.noHandlerFoundForExtension("foo.bar.Ext") assert(ex.getCondition.contains("NO_HANDLER_FOR_EXTENSION")) assert(ex.getMessage.contains("foo.bar.Ext")) - assert(ex.getSqlState == "XXSC1") + assert(ex.getSqlState == "42000") } test("notFoundCachedLocalRelation") { @@ -152,7 +152,7 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { assert(ex.getCondition.contains("NOT_FOUND_CACHED_LOCAL_RELATION")) assert(ex.getMessage.contains("abc123")) assert(ex.getMessage.contains("sess-uuid")) - assert(ex.getSqlState == "XXSC1") + assert(ex.getSqlState == "42000") } test("localRelationSizeLimitExceeded") { @@ -160,14 +160,14 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { assert(ex.getCondition.contains("LOCAL_RELATION_SIZE_LIMIT_EXCEEDED")) assert(ex.getMessage.contains("1000")) assert(ex.getMessage.contains("500")) - assert(ex.getSqlState == "XXSC1") + assert(ex.getSqlState == "42000") } test("functionEvalTypeNotSupported") { val ex = InvalidInputErrors.functionEvalTypeNotSupported(42) assert(ex.getCondition.contains("FUNCTION_EVAL_TYPE_NOT_SUPPORTED")) assert(ex.getMessage.contains("42")) - assert(ex.getSqlState == "XXSC1") + assert(ex.getSqlState == "42000") } test("streamingQueryRunIdMismatch") { @@ -176,7 +176,7 @@ class InvalidInputErrorsSuite extends PlanTest with SparkConnectPlanTest { assert(ex.getMessage.contains("q1")) assert(ex.getMessage.contains("run1")) assert(ex.getMessage.contains("run2")) - assert(ex.getSqlState == "XXSC1") + assert(ex.getSqlState == "42000") } // Helper case class to define test cases From f8aac0a8dd8aef437c8c40f9ce617263c0eb7ad4 Mon Sep 17 00:00:00 2001 From: Garland Zhang Date: Wed, 11 Mar 2026 15:48:08 +0000 Subject: [PATCH 4/5] Address review: use existing CONNECT_INVALID_PLAN (56K00) instead of new error class - Move all 66 subclasses under existing CONNECT_INVALID_PLAN (sqlState: 56K00) instead of the new SPARK_CONNECT_INVALID_PLAN_INPUT (sqlState: 42000) per suggestion from xi-db and agreement from garlandz-db - Add UNRESOLVED_COLUMN_AMONG_FIELD_NAMES as CONNECT_INVALID_PLAN subclass (was bare error key, per khakhlyuk comment) - Add INVALID_SCHEMA_NON_STRUCT_TYPE as CONNECT_INVALID_PLAN subclass (was reusing INVALID_SCHEMA.NON_STRUCT_TYPE, per khakhlyuk comment) - Update all callers across 6 files: SPARK_CONNECT_INVALID_PLAN_INPUT.* -> CONNECT_INVALID_PLAN.* - Update tests: SQL state assertions 42000 -> 56K00 Co-Authored-By: Claude Sonnet 4.6 --- .../resources/error/error-conditions.json | 5948 +++++++++-------- .../common/DataTypeProtoConverter.scala | 8 +- .../common/LiteralValueProtoConverter.scala | 20 +- .../connect/planner/InvalidInputErrors.scala | 102 +- .../sql/connect/service/SessionHolder.scala | 2 +- .../service/SparkConnectAnalyzeHandler.scala | 2 +- .../planner/InvalidInputErrorsSuite.scala | 20 +- 7 files changed, 3052 insertions(+), 3050 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 12b697ea72eac..bb2d154ea6a51 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -936,4553 +936,4883 @@ "To disable plan compression, set 'spark.connect.session.planCompression.threshold' to -1." ] }, - "PLAN_SIZE_LARGER_THAN_MAX" : { - "message" : [ - "The plan size is larger than max ( vs. )", - "This typically occurs when building very complex queries with many operations, large literals, or deeply nested expressions.", - "Consider splitting the query into smaller parts using temporary views for intermediate results or reducing the number of operations." - ] - } - }, - "sqlState" : "56K00" - }, - "CONNECT_ML" : { - "message" : [ - "Generic Spark Connect ML error." - ], - "subClass" : { - "ATTRIBUTE_NOT_ALLOWED" : { + "AGGREGATE_NEEDS_PLAN_INPUT" : { "message" : [ - " in is not allowed to be accessed." + "Aggregate needs a plan input" ] }, - "CACHE_INVALID" : { + "AGGREGATE_WITH_PIVOT_REQUIRES_PIVOT" : { "message" : [ - "Cannot retrieve Summary object from the ML cache.", - "Because the Summary object is evicted if you don't use it for more than minutes.", - "In this case, you can call `model.evaluate(dataset)` to create a new Summary object." + "Aggregate with GROUP_TYPE_PIVOT requires a Pivot" ] }, - "ML_CACHE_SIZE_OVERFLOW_EXCEPTION" : { + "ALIAS_WITH_MULTIPLE_IDENTIFIERS_AND_METADATA" : { "message" : [ - "The model cache size in current session is about to exceed", - " bytes.", - "Please delete existing cached model by executing 'del model' in python client before fitting new model or loading new model" + "Alias expressions with more than 1 identifier must not use optional metadata." ] }, - "MODEL_SIZE_OVERFLOW_EXCEPTION" : { + "ARRAY_LITERAL_MISSING_DATA_TYPE" : { "message" : [ - "The fitted or loaded model size is about bytes.", - "Please fit or load a model smaller than bytes." + "Data type information is missing in the array literal." ] }, - "MODEL_SUMMARY_LOST" : { + "ARRAY_LITERAL_NOT_SET" : { "message" : [ - "The model summary is lost because the cached model is offloaded." + "Array literal is not set." ] }, - "UNSUPPORTED_EXCEPTION" : { + "ASSERTION_FAILURE" : { "message" : [ "" ] - } - }, - "sqlState" : "XX000" - }, - "CONSTRAINT_ALREADY_EXISTS" : { - "message" : [ - "Constraint '' already exists. Please delete the existing constraint first.", - "Existing constraint:", - "" - ], - "sqlState" : "42710" - }, - "CONSTRAINT_DOES_NOT_EXIST" : { - "message" : [ - "Cannot drop nonexistent constraint from table ." - ], - "sqlState" : "42704" - }, - "CONVERSION_INVALID_INPUT" : { - "message" : [ - "The value () cannot be converted to because it is malformed. Correct the value as per the syntax, or change its format. Use to tolerate malformed input and return NULL instead." - ], - "sqlState" : "22018" - }, - "CORRUPTED_CATALOG_FUNCTION" : { - "message" : [ - "Cannot convert the catalog function '' into a SQL function due to corrupted function information in catalog. If the function is not a SQL function, please make sure the class name '' is loadable." - ], - "sqlState" : "0A000" - }, - "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS" : { - "message" : [ - "Not allowed to create the permanent view without explicitly assigning an alias for the expression ." - ], - "sqlState" : "0A000" - }, - "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE" : { - "message" : [ - "CREATE TABLE column specifies descriptor \"\" more than once, which is invalid." - ], - "sqlState" : "42710" - }, - "CREATE_VIEW_COLUMN_ARITY_MISMATCH" : { - "message" : [ - "Cannot create view , the reason is" - ], - "subClass" : { - "NOT_ENOUGH_DATA_COLUMNS" : { + }, + "CANNOT_FIND_CACHED_LOCAL_RELATION" : { "message" : [ - "not enough data columns:", - "View columns: .", - "Data columns: ." + "Cannot find a cached local relation for hash: " ] }, - "TOO_MANY_DATA_COLUMNS" : { + "CHUNKED_CACHED_LOCAL_RELATION_WITHOUT_DATA" : { "message" : [ - "too many data columns:", - "View columns: .", - "Data columns: ." + "ChunkedCachedLocalRelation should contain data." ] - } - }, - "sqlState" : "21S01" - }, - "CREATE_VIEW_WITH_IF_NOT_EXISTS_AND_REPLACE" : { - "message" : [ - "CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed." - ], - "sqlState" : "42601" - }, - "CURSOR_ALREADY_EXISTS" : { - "message" : [ - "Cannot declare cursor because it already exists in the current scope." - ], - "sqlState" : "42723" - }, - "CURSOR_ALREADY_OPEN" : { - "message" : [ - "Cannot open cursor because it is already open." - ], - "sqlState" : "24502" - }, - "CURSOR_NOT_FOUND" : { - "message" : [ - "Cursor not found in the current scope." - ], - "sqlState" : "42883" - }, - "CURSOR_NOT_OPEN" : { - "message" : [ - "Cannot fetch from or close cursor because it is not open." - ], - "sqlState" : "24501" - }, - "CURSOR_NO_MORE_ROWS" : { - "message" : [ - "No more rows available to fetch from cursor ." - ], - "sqlState" : "02000" - }, - "CURSOR_OUTSIDE_SCRIPT" : { - "message" : [ - "Cursor operations can only be used within SQL scripts." - ], - "sqlState" : "0A000" - }, - "CURSOR_REFERENCE_INVALID_QUALIFIER" : { - "message" : [ - "Cursor reference is invalid. Cursor references can only have at most one qualifier (e.g., label.cursor)." - ], - "sqlState" : "42601" - }, - "CYCLIC_FUNCTION_REFERENCE" : { - "message" : [ - "Cyclic function reference detected: ." - ], - "sqlState" : "42887" - }, - "DATAFLOW_GRAPH_NOT_FOUND" : { - "message" : [ - "Dataflow graph with id could not be found" - ], - "sqlState" : "KD011" - }, - "DATATYPE_CANNOT_ORDER" : { - "message" : [ - "Type does not support ordered operations." - ], - "sqlState" : "0A000" - }, - "DATATYPE_MISMATCH" : { - "message" : [ - "Cannot resolve due to data type mismatch:" - ], - "subClass" : { - "ARRAY_FUNCTION_DIFF_TYPES" : { + }, + "DATAFRAME_NOT_FOUND" : { "message" : [ - "Input to should have been followed by a value with same element type, but it's [, ]." + "No DataFrame with id is found in the session " ] }, - "BINARY_ARRAY_DIFF_TYPES" : { + "DATA_TYPE_UNSUPPORTED_CATALYST_TO_PROTO" : { "message" : [ - "Input to function should have been two with same element type, but it's [, ]." + "Does not support convert to connect proto types." ] }, - "BINARY_OP_DIFF_TYPES" : { + "DATA_TYPE_UNSUPPORTED_PROTO_TO_CATALYST" : { "message" : [ - "the left and right operands of the binary operator have incompatible types ( and )." + "Does not support convert to catalyst types." ] }, - "BINARY_OP_WRONG_TYPE" : { + "DEDUPLICATE_ALL_COLUMNS_AND_SUBSET" : { "message" : [ - "the binary operator requires the input type , not ." + "Cannot deduplicate on both all columns and a subset of columns" ] }, - "BLOOM_FILTER_BINARY_OP_WRONG_TYPE" : { + "DEDUPLICATE_NEEDS_INPUT" : { "message" : [ - "The Bloom filter binary input to should be either a constant value or a scalar subquery expression, but it's ." + "Deduplicate needs a plan input" ] }, - "BLOOM_FILTER_WRONG_TYPE" : { + "DEDUPLICATE_REQUIRES_COLUMNS_OR_ALL" : { "message" : [ - "Input to function should have been followed by value with , but it's []." + "Deduplicate requires to either deduplicate on all columns or a subset of columns" ] }, - "CANNOT_CONVERT_TO_JSON" : { + "EXCEPT_DOES_NOT_SUPPORT_UNION_BY_NAME" : { "message" : [ - "Unable to convert column of type to JSON." + "Except does not support union_by_name" ] }, - "CANNOT_DROP_ALL_FIELDS" : { + "EXPECTED_NULL_VALUE" : { "message" : [ - "Cannot drop all fields in struct." + "Expected null value, but got " ] }, - "CAST_WITHOUT_SUGGESTION" : { + "EXPECTING_SCALA_UDF" : { "message" : [ - "cannot cast to ." + "Expecting a Scala UDF, but get " ] }, - "CAST_WITH_CONF_SUGGESTION" : { + "FIELD_CANNOT_BE_EMPTY" : { "message" : [ - "cannot cast to with ANSI mode on.", - "If you have to cast to , you can set as ." + " in cannot be empty" ] }, - "CAST_WITH_FUNC_SUGGESTION" : { + "FUNCTION_EVAL_TYPE_NOT_SUPPORTED" : { "message" : [ - "cannot cast to .", - "To convert values from to , you can use the functions instead." + "Function with EvalType: is not supported" ] }, - "CREATE_MAP_KEY_DIFF_TYPES" : { + "GROUPING_EXPRESSION_ABSENT" : { "message" : [ - "The given keys of function should all be the same type, but they are ." + "The grouping expression cannot be absent for KeyValueGroupedDataset" ] }, - "CREATE_MAP_VALUE_DIFF_TYPES" : { + "INCOMPATIBLE_LITERAL_DATA_TYPE" : { "message" : [ - "The given values of function should all be the same type, but they are ." + "Incompatible data type for literal " ] }, - "CREATE_NAMED_STRUCT_WITHOUT_FOLDABLE_STRING" : { + "INPUT_DATA_NO_SCHEMA" : { "message" : [ - "Only foldable `STRING` expressions are allowed to appear at odd position, but they are ." + "Input data for LocalRelation does not produce a schema." ] }, - "DATA_DIFF_TYPES" : { + "INTERSECT_DOES_NOT_SUPPORT_UNION_BY_NAME" : { "message" : [ - "Input to should all be the same type, but it's ." + "Intersect does not support union_by_name" ] }, - "FILTER_NOT_BOOLEAN" : { + "INVALID_ENUM" : { "message" : [ - "Filter expression of type is not a boolean." + "This enum value of is invalid: ()" ] }, - "HASH_MAP_TYPE" : { + "INVALID_JDBC_PARAMS" : { "message" : [ - "Input to the function cannot contain elements of the \"MAP\" type. In Spark, same maps may have different hashcode, thus hash expressions are prohibited on \"MAP\" elements. To restore previous behavior set \"spark.sql.legacy.allowHashOnMapType\" to \"true\"." + "Invalid jdbc params, please specify jdbc url and table." ] }, - "HASH_VARIANT_TYPE" : { + "INVALID_ONE_OF_FIELD_NOT_SET" : { "message" : [ - "Input to the function cannot contain elements of the \"VARIANT\" type yet." + "This oneOf field in is not set: " ] }, - "INPUT_SIZE_NOT_ONE" : { + "INVALID_ONE_OF_FIELD_NOT_SUPPORTED" : { "message" : [ - "Length of should be 1." + "This oneOf field message in is not supported: ()" ] }, - "INVALID_ARG_VALUE" : { + "INVALID_SCHEMA_NON_STRUCT_TYPE" : { "message" : [ - "The value must to be a literal of , but got ." + "The input schema is not a struct type, but got ." ] }, - "INVALID_JSON_MAP_KEY_TYPE" : { + "INVALID_SQL_WITH_REFERENCES" : { "message" : [ - "Input schema can only contain STRING as a key type for a MAP." + " is not a valid relation for SQL with references" ] }, - "INVALID_JSON_SCHEMA" : { + "INVALID_WITH_RELATION_REFERENCE" : { "message" : [ - "Input schema must be a struct, an array, a map or a variant." + "Invalid WithRelation reference" ] }, - "INVALID_MAP_KEY_TYPE" : { + "LAMBDA_FUNCTION_ARGUMENT_COUNT_INVALID" : { "message" : [ - "The key of map cannot be/contain ." + "LambdaFunction requires 1 ~ 3 arguments, but got ones!" ] }, - "INVALID_ORDERING_TYPE" : { + "LOCAL_RELATION_CHUNK_SIZE_LIMIT_EXCEEDED" : { "message" : [ - "The does not support ordering on type ." + "One of cached local relation chunks exceeded the limit of bytes." ] }, - "INVALID_ROW_LEVEL_OPERATION_ASSIGNMENTS" : { + "LOCAL_RELATION_SIZE_LIMIT_EXCEEDED" : { "message" : [ - "" + "Cached local relation size ( bytes) exceeds the limit ( bytes)." ] }, - "INVALID_XML_MAP_KEY_TYPE" : { + "LOWER_BOUND_REQUIRED_IN_WINDOW_FRAME" : { "message" : [ - "Input schema can only contain STRING as a key type for a MAP." + "LowerBound is required in WindowFrame" ] }, - "INVALID_XML_SCHEMA" : { + "MAP_LITERAL_MISSING_DATA_TYPE" : { "message" : [ - "Input schema must be a struct or a variant." + "Data type information is missing in the map literal." ] }, - "IN_SUBQUERY_DATA_TYPE_MISMATCH" : { + "MAP_LITERAL_NOT_SET" : { "message" : [ - "The data type of one or more elements in the left hand side of an IN subquery is not compatible with the data type of the output of the subquery. Mismatched columns: [], left side: [], right side: []." + "Map literal is not set." ] }, - "IN_SUBQUERY_LENGTH_MISMATCH" : { + "MULTIPLE_PATHS_NOT_SUPPORTED_FOR_STREAMING_SOURCE" : { "message" : [ - "The number of columns in the left hand side of an IN subquery does not match the number of columns in the output of subquery. Left hand side columns(length: ): [], right hand side columns(length: ): []." + "Multiple paths are not supported for streaming source" ] }, - "MAP_CONCAT_DIFF_TYPES" : { + "NA_FILL_VALUES_EMPTY" : { "message" : [ - "The should all be of type map, but it's ." + "values must contains at least 1 item!" ] }, - "MAP_FUNCTION_DIFF_TYPES" : { + "NA_FILL_VALUES_LENGTH_MISMATCH" : { "message" : [ - "Input to should have been followed by a value with same key type, but it's [, ]." + "When values contains more than 1 items, values and cols should have the same length!" ] }, - "MAP_ZIP_WITH_DIFF_TYPES" : { + "NOT_FOUND_CACHED_LOCAL_RELATION" : { "message" : [ - "Input to the should have been two maps with compatible key types, but it's [, ]." + "Not found any cached local relation with the hash: in the session with sessionUUID ." ] }, - "NON_FOLDABLE_INPUT" : { + "NOT_FOUND_CHUNKED_CACHED_LOCAL_RELATION" : { "message" : [ - "the input should be a foldable expression; however, got ." + "Not found chunked cached local relation block with the hash: in the session with sessionUUID ." ] }, - "NON_STRING_TYPE" : { + "NO_HANDLER_FOR_EXTENSION" : { "message" : [ - "all arguments of the function must be strings." + "No handler found for extension type: " ] }, - "NON_STRUCT_TYPE" : { + "PREDICATES_NOT_SUPPORTED_FOR_DATA_SOURCE" : { "message" : [ - "the input should be a struct expression; however, got ." + "Predicates are not supported for data sources." ] }, - "NULL_TYPE" : { + "PYTHON_UDT_MISSING_FIELDS" : { "message" : [ - "Null typed values cannot be used as arguments of ." + "PythonUserDefinedType requires all the three fields: python_class, serialized_python_class and sql_type." ] }, - "PARAMETER_CONSTRAINT_VIOLATION" : { + "REDUCE_SHOULD_CARRY_SCALAR_SCALA_UDF" : { "message" : [ - "The () must be the ()." + "reduce should carry a scalar scala udf, but got " ] }, - "RANGE_FRAME_INVALID_TYPE" : { + "ROW_NOT_SUPPORTED_FOR_UDF" : { "message" : [ - "The data type used in the order specification does not support the data type which is used in the range frame." + "Row is not a supported type for this UDF." ] }, - "RANGE_FRAME_MULTI_ORDER" : { + "SCHEMA_REQUIRED_FOR_LOCAL_RELATION" : { "message" : [ - "A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: ." + "Schema for LocalRelation is required when the input data is not provided." ] }, - "RANGE_FRAME_WITHOUT_ORDER" : { + "SET_OPERATION_MUST_HAVE_TWO_INPUTS" : { "message" : [ - "A range window frame cannot be used in an unordered window specification." + "Set operation must have 2 inputs" ] }, - "SEQUENCE_WRONG_INPUT_TYPES" : { + "SQL_COMMAND_EXPECTS_SQL_OR_WITH_RELATIONS" : { "message" : [ - " uses the wrong parameter type. The parameter type must conform to:", - "1. The start and stop expressions must resolve to the same type.", - "2. If start and stop expressions resolve to the type, then the step expression must resolve to the type.", - "3. Otherwise, if start and stop expressions resolve to the type, then the step expression must resolve to the same type." + "SQL command expects either a SQL or a WithRelations, but got " ] }, - "SPECIFIED_WINDOW_FRAME_DIFF_TYPES" : { + "STREAMING_QUERY_NOT_FOUND" : { "message" : [ - "Window frame bounds and do not have the same type: <> ." + "Streaming query is not found" ] }, - "SPECIFIED_WINDOW_FRAME_INVALID_BOUND" : { + "STREAMING_QUERY_RUN_ID_MISMATCH" : { "message" : [ - "Window frame upper bound does not follow the lower bound ." + "Run id mismatch for query id . Run id in the request does not match one on the server . The query might have restarted." ] }, - "SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE" : { + "STRUCT_LITERAL_MISSING_DATA_TYPE" : { "message" : [ - "The data type of the bound does not match the expected data type ." + "Data type information is missing in the struct literal." ] }, - "SPECIFIED_WINDOW_FRAME_WITHOUT_FOLDABLE" : { + "STRUCT_LITERAL_NOT_SET" : { "message" : [ - "Window frame bound is not a literal." + "Struct literal is not set." ] }, - "SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON" : { + "UDT_TYPE_FIELD_INVALID" : { "message" : [ - "The lower bound of a window frame must be to the upper bound." + "UserDefinedType requires the 'type' field to be 'udt', but got ''." ] }, - "STACK_COLUMN_DIFF_TYPES" : { + "UNION_BY_NAME_ALLOW_MISSING_COL_REQUIRES_BY_NAME" : { "message" : [ - "The data type of the column () do not have the same type: () <> ()." + "UnionByName `allowMissingCol` can be true only if `byName` is true." ] }, - "TYPE_CHECK_FAILURE_WITH_HINT" : { + "UNKNOWN_ANALYZE_METHOD" : { "message" : [ - "." + "Unknown Analyze Method !" ] }, - "UNEXPECTED_CLASS_TYPE" : { + "UNRESOLVED_COLUMN_AMONG_FIELD_NAMES" : { "message" : [ - "class not found." + "Cannot resolve column name \"\" among ()." ] }, - "UNEXPECTED_INPUT_TYPE" : { + "UNRESOLVED_NAMED_LAMBDA_VARIABLE_REQUIRES_NAME_PART" : { "message" : [ - "The parameter requires the type, however has the type ." + "UnresolvedNamedLambdaVariable requires at least one name part!" ] }, - "UNEXPECTED_NULL" : { + "UNRESOLVED_STAR_TARGET_INVALID" : { "message" : [ - "The must not be null." + "UnresolvedStar requires a unparsed target ending with '.*', but got ." ] }, - "UNEXPECTED_RETURN_TYPE" : { + "UNRESOLVED_STAR_WITH_BOTH_TARGET_AND_PLAN_ID" : { "message" : [ - "The requires return type, but the actual is type." + "UnresolvedStar with both target and plan id is not supported." ] }, - "UNEXPECTED_STATIC_METHOD" : { + "UNSUPPORTED_LITERAL_TYPE" : { "message" : [ - "cannot find a static method that matches the argument types in ." + "Unsupported Literal Type: " ] }, - "UNSUPPORTED_INPUT_TYPE" : { + "UNSUPPORTED_USER_DEFINED_FUNCTION_IMPLEMENTATION" : { "message" : [ - "The input of can't be type data." + "Unsupported UserDefinedFunction implementation: " ] }, - "VALUE_OUT_OF_RANGE" : { + "UPPER_BOUND_REQUIRED_IN_WINDOW_FRAME" : { "message" : [ - "The must be between (current value = )." + "UpperBound is required in WindowFrame" ] }, - "WRONG_NUM_ARG_TYPES" : { + "USING_COLUMNS_OR_JOIN_CONDITION_SET_IN_JOIN" : { "message" : [ - "The expression requires argument types but the actual number is ." + "Using columns or join conditions cannot be set at the same time in Join" ] }, - "WRONG_NUM_ENDPOINTS" : { + "WINDOW_FUNCTION_REQUIRED" : { "message" : [ - "The number of endpoints must be >= 2 to construct intervals but the actual number is ." + "WindowFunction is required in WindowExpression" + ] + }, + "WITH_COLUMNS_REQUIRE_SINGLE_NAME_PART" : { + "message" : [ + "WithColumns require column name only contains one name part, but got " + ] + }, + "PLAN_SIZE_LARGER_THAN_MAX" : { + "message" : [ + "The plan size is larger than max ( vs. )", + "This typically occurs when building very complex queries with many operations, large literals, or deeply nested expressions.", + "Consider splitting the query into smaller parts using temporary views for intermediate results or reducing the number of operations." ] } }, - "sqlState" : "42K09" + "sqlState" : "56K00" }, - "DATATYPE_MISSING_SIZE" : { + "CONNECT_ML" : { "message" : [ - "DataType requires a length parameter, for example (10). Please specify the length." + "Generic Spark Connect ML error." ], - "sqlState" : "42K01" + "subClass" : { + "ATTRIBUTE_NOT_ALLOWED" : { + "message" : [ + " in is not allowed to be accessed." + ] + }, + "CACHE_INVALID" : { + "message" : [ + "Cannot retrieve Summary object from the ML cache.", + "Because the Summary object is evicted if you don't use it for more than minutes.", + "In this case, you can call `model.evaluate(dataset)` to create a new Summary object." + ] + }, + "ML_CACHE_SIZE_OVERFLOW_EXCEPTION" : { + "message" : [ + "The model cache size in current session is about to exceed", + " bytes.", + "Please delete existing cached model by executing 'del model' in python client before fitting new model or loading new model" + ] + }, + "MODEL_SIZE_OVERFLOW_EXCEPTION" : { + "message" : [ + "The fitted or loaded model size is about bytes.", + "Please fit or load a model smaller than bytes." + ] + }, + "MODEL_SUMMARY_LOST" : { + "message" : [ + "The model summary is lost because the cached model is offloaded." + ] + }, + "UNSUPPORTED_EXCEPTION" : { + "message" : [ + "" + ] + } + }, + "sqlState" : "XX000" }, - "DATA_SOURCE_ALREADY_EXISTS" : { + "CONSTRAINT_ALREADY_EXISTS" : { "message" : [ - "Data source '' already exists. Please choose a different name for the new data source." + "Constraint '' already exists. Please delete the existing constraint first.", + "Existing constraint:", + "" ], "sqlState" : "42710" }, - "DATA_SOURCE_EXTERNAL_ERROR" : { + "CONSTRAINT_DOES_NOT_EXIST" : { "message" : [ - "Encountered error when saving to external data source." + "Cannot drop nonexistent constraint from table ." ], - "sqlState" : "KD010" + "sqlState" : "42704" }, - "DATA_SOURCE_NOT_EXIST" : { + "CONVERSION_INVALID_INPUT" : { "message" : [ - "Data source '' not found. Please make sure the data source is registered." + "The value () cannot be converted to because it is malformed. Correct the value as per the syntax, or change its format. Use to tolerate malformed input and return NULL instead." ], - "sqlState" : "42704" + "sqlState" : "22018" }, - "DATA_SOURCE_NOT_FOUND" : { + "CORRUPTED_CATALOG_FUNCTION" : { "message" : [ - "Failed to find the data source: . Make sure the provider name is correct and the package is properly registered and compatible with your Spark version." + "Cannot convert the catalog function '' into a SQL function due to corrupted function information in catalog. If the function is not a SQL function, please make sure the class name '' is loadable." ], - "sqlState" : "42K02" + "sqlState" : "0A000" }, - "DATA_SOURCE_TABLE_SCHEMA_MISMATCH" : { + "CREATE_PERMANENT_VIEW_WITHOUT_ALIAS" : { "message" : [ - "The schema of the data source table does not match the expected schema. If you are using the DataFrameReader.schema API or creating a table, avoid specifying the schema.", - "Data Source schema: ", - "Expected schema: " + "Not allowed to create the permanent view without explicitly assigning an alias for the expression ." ], - "sqlState" : "42K03" + "sqlState" : "0A000" }, - "DATETIME_FIELD_OUT_OF_BOUNDS" : { + "CREATE_TABLE_COLUMN_DESCRIPTOR_DUPLICATE" : { "message" : [ - "." + "CREATE TABLE column specifies descriptor \"\" more than once, which is invalid." + ], + "sqlState" : "42710" + }, + "CREATE_VIEW_COLUMN_ARITY_MISMATCH" : { + "message" : [ + "Cannot create view , the reason is" ], "subClass" : { - "WITHOUT_SUGGESTION" : { + "NOT_ENOUGH_DATA_COLUMNS" : { "message" : [ - "" + "not enough data columns:", + "View columns: .", + "Data columns: ." ] }, - "WITH_SUGGESTION" : { + "TOO_MANY_DATA_COLUMNS" : { "message" : [ - "If necessary set to \"false\" to bypass this error." + "too many data columns:", + "View columns: .", + "Data columns: ." ] } }, - "sqlState" : "22023" - }, - "DATETIME_OVERFLOW" : { - "message" : [ - "Datetime operation overflow: ." - ], - "sqlState" : "22008" + "sqlState" : "21S01" }, - "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION" : { + "CREATE_VIEW_WITH_IF_NOT_EXISTS_AND_REPLACE" : { "message" : [ - "Decimal precision exceeds max precision ." + "CREATE VIEW with both IF NOT EXISTS and REPLACE is not allowed." ], - "sqlState" : "22003" + "sqlState" : "42601" }, - "DEFAULT_DATABASE_NOT_EXISTS" : { + "CURSOR_ALREADY_EXISTS" : { "message" : [ - "Default database does not exist, please create it first or change default database to ``." + "Cannot declare cursor because it already exists in the current scope." ], - "sqlState" : "42704" + "sqlState" : "42723" }, - "DEFAULT_PLACEMENT_INVALID" : { + "CURSOR_ALREADY_OPEN" : { "message" : [ - "A DEFAULT keyword in a MERGE, INSERT, UPDATE, or SET VARIABLE command could not be directly assigned to a target column because it was part of an expression.", - "For example: `UPDATE SET c1 = DEFAULT` is allowed, but `UPDATE T SET c1 = DEFAULT + 1` is not allowed." + "Cannot open cursor because it is already open." ], - "sqlState" : "42608" + "sqlState" : "24502" }, - "DEFAULT_UNSUPPORTED" : { + "CURSOR_NOT_FOUND" : { "message" : [ - "Failed to execute command because DEFAULT values are not supported for target data source with table provider: \"\"." + "Cursor not found in the current scope." ], - "sqlState" : "42623" + "sqlState" : "42883" }, - "DEFINE_FLOW_ONCE_OPTION_NOT_SUPPORTED" : { + "CURSOR_NOT_OPEN" : { "message" : [ - "Defining a one-time flow with the 'once' option is not supported." + "Cannot fetch from or close cursor because it is not open." ], - "sqlState" : "0A000" + "sqlState" : "24501" }, - "DESCRIBE_JSON_NOT_EXTENDED" : { + "CURSOR_NO_MORE_ROWS" : { "message" : [ - "DESCRIBE TABLE ... AS JSON only supported when [EXTENDED|FORMATTED] is specified.", - "For example: DESCRIBE EXTENDED AS JSON is supported but DESCRIBE AS JSON is not." + "No more rows available to fetch from cursor ." ], - "sqlState" : "0A000" + "sqlState" : "02000" }, - "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED" : { + "CURSOR_OUTSIDE_SCRIPT" : { "message" : [ - "Distinct window functions are not supported: ." + "Cursor operations can only be used within SQL scripts." ], "sqlState" : "0A000" }, - "DIVIDE_BY_ZERO" : { - "message" : [ - "Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set to \"false\" to bypass this error." - ], - "sqlState" : "22012" - }, - "DUPLICATED_CTE_NAMES" : { - "message" : [ - "CTE definition can't have duplicate names: ." - ], - "sqlState" : "42602" - }, - "DUPLICATED_FIELD_NAME_IN_ARROW_STRUCT" : { - "message" : [ - "Duplicated field names in Arrow Struct are not allowed, got ." - ], - "sqlState" : "42713" - }, - "DUPLICATED_MAP_KEY" : { - "message" : [ - "Duplicate map key was found, please check the input data.", - "If you want to remove the duplicated keys, you can set to \"LAST_WIN\" so that the key inserted at last takes precedence." - ], - "sqlState" : "23505" - }, - "DUPLICATED_METRICS_NAME" : { + "CURSOR_REFERENCE_INVALID_QUALIFIER" : { "message" : [ - "The metric name is not unique: . The same name cannot be used for metrics with different results.", - "However multiple instances of metrics with with same result and name are allowed (e.g. self-joins)." + "Cursor reference is invalid. Cursor references can only have at most one qualifier (e.g., label.cursor)." ], - "sqlState" : "42710" + "sqlState" : "42601" }, - "DUPLICATE_ASSIGNMENTS" : { + "CYCLIC_FUNCTION_REFERENCE" : { "message" : [ - "The columns or variables appear more than once as assignment targets." + "Cyclic function reference detected: ." ], - "sqlState" : "42701" + "sqlState" : "42887" }, - "DUPLICATE_CLAUSES" : { + "DATAFLOW_GRAPH_NOT_FOUND" : { "message" : [ - "Found duplicate clauses: . Please, remove one of them." + "Dataflow graph with id could not be found" ], - "sqlState" : "42614" + "sqlState" : "KD011" }, - "DUPLICATE_CONDITION_IN_SCOPE" : { + "DATATYPE_CANNOT_ORDER" : { "message" : [ - "Found duplicate condition in the scope. Please, remove one of them." + "Type does not support ordered operations." ], - "sqlState" : "42734" + "sqlState" : "0A000" }, - "DUPLICATE_EXCEPTION_HANDLER" : { + "DATATYPE_MISMATCH" : { "message" : [ - "Found duplicate handlers. Please, remove one of them." + "Cannot resolve due to data type mismatch:" ], "subClass" : { - "CONDITION" : { + "ARRAY_FUNCTION_DIFF_TYPES" : { "message" : [ - "Found duplicate handlers for the same condition ." + "Input to should have been followed by a value with same element type, but it's [, ]." ] }, - "SQLSTATE" : { + "BINARY_ARRAY_DIFF_TYPES" : { "message" : [ - "Found duplicate handlers for the same SQLSTATE ." + "Input to function should have been two with same element type, but it's [, ]." ] - } - }, - "sqlState" : "42734" - }, - "DUPLICATE_FLOW_SQL_CONF" : { - "message" : [ - "Found duplicate sql conf for dataset '': '' is defined by both '' and ''" - ], - "sqlState" : "42710" - }, - "DUPLICATE_KEY" : { - "message" : [ - "Found duplicate keys ." - ], - "sqlState" : "23505" - }, - "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT" : { - "message" : [ - "Call to routine is invalid because it includes multiple argument assignments to the same parameter name ." - ], - "subClass" : { - "BOTH_POSITIONAL_AND_NAMED" : { + }, + "BINARY_OP_DIFF_TYPES" : { "message" : [ - "A positional argument and named argument both referred to the same parameter. Please remove the named argument referring to this parameter." + "the left and right operands of the binary operator have incompatible types ( and )." ] }, - "DOUBLE_NAMED_ARGUMENT_REFERENCE" : { + "BINARY_OP_WRONG_TYPE" : { "message" : [ - "More than one named argument referred to the same parameter. Please assign a value only once." + "the binary operator requires the input type , not ." ] - } - }, - "sqlState" : "4274K" - }, - "DUPLICATE_ROUTINE_PARAMETER_NAMES" : { - "message" : [ - "Found duplicate name(s) in the parameter list of the user-defined routine : ." - ], - "sqlState" : "42734" - }, - "DUPLICATE_ROUTINE_RETURNS_COLUMNS" : { - "message" : [ - "Found duplicate column(s) in the RETURNS clause column list of the user-defined routine : ." - ], - "sqlState" : "42711" - }, - "DUPLICATE_VARIABLE_NAME_INSIDE_DECLARE" : { - "message" : [ - "Found duplicate variable in the declare variable list. Please, remove one of them." - ], - "sqlState" : "42734" - }, - "DYNAMIC_PARTITION_WRITE_PARTITION_NUM_LIMIT_EXCEEDED" : { - "message" : [ - "Number of dynamic partitions created is , which is more than . To solve this try to set to at least ." - ], - "sqlState" : "54054" - }, - "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : { - "message" : [ - "Previous node emitted a row with eventTime= which is older than current_watermark_value=", - "This can lead to correctness issues in the stateful operators downstream in the execution pipeline.", - "Please correct the operator logic to emit rows after current global watermark value." - ], - "sqlState" : "42815" - }, - "EMPTY_JSON_FIELD_VALUE" : { - "message" : [ - "Failed to parse an empty string for data type ." - ], - "sqlState" : "42604" - }, - "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE" : { - "message" : [ - "The datasource does not support writing empty or nested empty schemas. Please make sure the data schema has at least one or more column(s)." - ], - "sqlState" : "0A000" - }, - "ENCODER_NOT_FOUND" : { - "message" : [ - "Not found an encoder of the type to Spark SQL internal representation.", - "Consider to change the input type to one of supported at '/sql-ref-datatypes.html'." - ], - "sqlState" : "42704" - }, - "END_LABEL_WITHOUT_BEGIN_LABEL" : { - "message" : [ - "End label can not exist without begin label." - ], - "sqlState" : "42K0L" - }, - "ERROR_READING_AVRO_UNKNOWN_FINGERPRINT" : { - "message" : [ - "Error reading avro data -- encountered an unknown fingerprint: , not sure what schema to use.", - "This could happen if you registered additional schemas after starting your spark context." - ], - "sqlState" : "KD00B" - }, - "EVENT_TIME_IS_NOT_ON_TIMESTAMP_TYPE" : { - "message" : [ - "The event time has the invalid type , but expected \"TIMESTAMP\"." - ], - "sqlState" : "42K09" - }, - "EVENT_TIME_MUST_BE_TOP_LEVEL_COLUMN" : { - "message" : [ - "The event time in withWatermark must be a top-level column, but '' is a nested field. To use it, alias it to a top-level column in a select before withWatermark." - ], - "sqlState" : "42K09" - }, - "EXCEED_LIMIT_LENGTH" : { - "message" : [ - "Exceeds char/varchar type length limitation: ." - ], - "sqlState" : "54006" - }, - "EXCEPT_NESTED_COLUMN_INVALID_TYPE" : { - "message" : [ - "EXCEPT column was resolved and expected to be StructType, but found type ." - ], - "sqlState" : "428H2" - }, - "EXCEPT_OVERLAPPING_COLUMNS" : { - "message" : [ - "Columns in an EXCEPT list must be distinct and non-overlapping, but got ()." - ], - "sqlState" : "42702" - }, - "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES" : { - "message" : [ - "The USING clause of this EXECUTE IMMEDIATE command contained multiple arguments with same alias (), which is invalid; please update the command to specify unique aliases and then try it again." - ], - "sqlState" : "42701" - }, - "EXPECT_PERMANENT_VIEW_NOT_TEMP" : { - "message" : [ - "'' expects a permanent view but is a temp view." - ], - "sqlState" : "42809" - }, - "EXPECT_TABLE_NOT_VIEW" : { - "message" : [ - "'' expects a table but is a view." - ], - "subClass" : { - "NO_ALTERNATIVE" : { + }, + "BLOOM_FILTER_BINARY_OP_WRONG_TYPE" : { "message" : [ - "" + "The Bloom filter binary input to should be either a constant value or a scalar subquery expression, but it's ." ] }, - "USE_ALTER_VIEW" : { + "BLOOM_FILTER_WRONG_TYPE" : { "message" : [ - "Please use ALTER VIEW instead." + "Input to function should have been followed by value with , but it's []." ] - } - }, - "sqlState" : "42809" - }, - "EXPECT_VIEW_NOT_TABLE" : { - "message" : [ - "The table does not support ." - ], - "subClass" : { - "NO_ALTERNATIVE" : { + }, + "CANNOT_CONVERT_TO_JSON" : { "message" : [ - "" + "Unable to convert column of type to JSON." ] }, - "USE_ALTER_TABLE" : { + "CANNOT_DROP_ALL_FIELDS" : { "message" : [ - "Please use ALTER TABLE instead." + "Cannot drop all fields in struct." ] - } - }, - "sqlState" : "42809" - }, - "EXPRESSION_DECODING_FAILED" : { - "message" : [ - "Failed to decode a row to a value of the expressions: ." - ], - "sqlState" : "42846" - }, - "EXPRESSION_ENCODING_FAILED" : { - "message" : [ - "Failed to encode a value of the expressions: to a row." - ], - "sqlState" : "42846" - }, - "EXPRESSION_TRANSLATION_TO_V2_IS_NOT_SUPPORTED" : { - "message" : [ - "Expression cannot be translated to v2 expression." - ], - "sqlState" : "0A000" - }, - "EXPRESSION_TYPE_IS_NOT_ORDERABLE" : { - "message" : [ - "Column expression cannot be sorted because its type is not orderable." - ], - "sqlState" : "42822" - }, - "FAILED_EXECUTE_UDF" : { - "message" : [ - "User defined function (: () => ) failed due to: ." - ], - "sqlState" : "39000" - }, - "FAILED_FUNCTION_CALL" : { - "message" : [ - "Failed preparing of the function for call. Please, double check function's arguments." - ], - "sqlState" : "38000" - }, - "FAILED_JDBC" : { - "message" : [ - "Failed JDBC on the operation:" - ], - "subClass" : { - "ALTER_TABLE" : { + }, + "CAST_WITHOUT_SUGGESTION" : { "message" : [ - "Alter the table ." + "cannot cast to ." ] }, - "CONNECTION" : { + "CAST_WITH_CONF_SUGGESTION" : { "message" : [ - "Couldn't connect to the database" + "cannot cast to with ANSI mode on.", + "If you have to cast to , you can set as ." ] }, - "CREATE_INDEX" : { + "CAST_WITH_FUNC_SUGGESTION" : { "message" : [ - "Create the index in the table." + "cannot cast to .", + "To convert values from to , you can use the functions instead." ] }, - "CREATE_NAMESPACE" : { + "CREATE_MAP_KEY_DIFF_TYPES" : { "message" : [ - "Create the namespace ." + "The given keys of function should all be the same type, but they are ." ] }, - "CREATE_NAMESPACE_COMMENT" : { + "CREATE_MAP_VALUE_DIFF_TYPES" : { "message" : [ - "Create a comment on the namespace: ." + "The given values of function should all be the same type, but they are ." ] }, - "CREATE_TABLE" : { + "CREATE_NAMED_STRUCT_WITHOUT_FOLDABLE_STRING" : { "message" : [ - "Create the table ." + "Only foldable `STRING` expressions are allowed to appear at odd position, but they are ." ] }, - "DROP_INDEX" : { + "DATA_DIFF_TYPES" : { "message" : [ - "Drop the index in the table." + "Input to should all be the same type, but it's ." ] }, - "DROP_NAMESPACE" : { + "FILTER_NOT_BOOLEAN" : { "message" : [ - "Drop the namespace ." + "Filter expression of type is not a boolean." ] }, - "DROP_TABLE" : { + "HASH_MAP_TYPE" : { "message" : [ - "Drop the table ." + "Input to the function cannot contain elements of the \"MAP\" type. In Spark, same maps may have different hashcode, thus hash expressions are prohibited on \"MAP\" elements. To restore previous behavior set \"spark.sql.legacy.allowHashOnMapType\" to \"true\"." ] }, - "GET_TABLES" : { + "HASH_VARIANT_TYPE" : { "message" : [ - "Get tables from the namespace: ." + "Input to the function cannot contain elements of the \"VARIANT\" type yet." ] }, - "LIST_NAMESPACES" : { + "INPUT_SIZE_NOT_ONE" : { "message" : [ - "List namespaces." + "Length of should be 1." ] }, - "LOAD_TABLE" : { + "INVALID_ARG_VALUE" : { "message" : [ - "Load the table ." + "The value must to be a literal of , but got ." ] }, - "NAMESPACE_EXISTS" : { + "INVALID_JSON_MAP_KEY_TYPE" : { "message" : [ - "Check that the namespace exists." + "Input schema can only contain STRING as a key type for a MAP." ] }, - "REMOVE_NAMESPACE_COMMENT" : { + "INVALID_JSON_SCHEMA" : { "message" : [ - "Remove a comment on the namespace: ." + "Input schema must be a struct, an array, a map or a variant." ] }, - "RENAME_TABLE" : { + "INVALID_MAP_KEY_TYPE" : { "message" : [ - "Rename the table to ." + "The key of map cannot be/contain ." ] }, - "TABLE_EXISTS" : { + "INVALID_ORDERING_TYPE" : { "message" : [ - "Check that the table exists." + "The does not support ordering on type ." ] }, - "UNCLASSIFIED" : { + "INVALID_ROW_LEVEL_OPERATION_ASSIGNMENTS" : { "message" : [ - "" + "" ] - } - }, - "sqlState" : "HV000" - }, - "FAILED_PARSE_STRUCT_TYPE" : { - "message" : [ - "Failed parsing struct: ." - ], - "sqlState" : "22018" - }, - "FAILED_READ_FILE" : { - "message" : [ - "Encountered error while reading file ." - ], - "subClass" : { - "CANNOT_READ_FILE_FOOTER" : { + }, + "INVALID_XML_MAP_KEY_TYPE" : { "message" : [ - "Could not read footer. Please ensure that the file is in either ORC or Parquet format.", - "If not, please convert it to a valid format. If the file is in the valid format, please check if it is corrupt.", - "If it is, you can choose to either ignore it or fix the corruption." + "Input schema can only contain STRING as a key type for a MAP." ] }, - "FILE_NOT_EXIST" : { + "INVALID_XML_SCHEMA" : { "message" : [ - "File does not exist. It is possible the underlying files have been updated.", - "You can explicitly invalidate the cache in Spark by running 'REFRESH TABLE tableName' command in SQL or by recreating the Dataset/DataFrame involved." + "Input schema must be a struct or a variant." ] }, - "NO_HINT" : { + "IN_SUBQUERY_DATA_TYPE_MISMATCH" : { "message" : [ - "" + "The data type of one or more elements in the left hand side of an IN subquery is not compatible with the data type of the output of the subquery. Mismatched columns: [], left side: [], right side: []." ] }, - "PARQUET_COLUMN_DATA_TYPE_MISMATCH" : { + "IN_SUBQUERY_LENGTH_MISMATCH" : { "message" : [ - "Data type mismatches when reading Parquet column . Expected Spark type , actual Parquet type ." + "The number of columns in the left hand side of an IN subquery does not match the number of columns in the output of subquery. Left hand side columns(length: ): [], right hand side columns(length: ): []." ] }, - "UNSUPPORTED_FILE_SYSTEM" : { + "MAP_CONCAT_DIFF_TYPES" : { "message" : [ - "The file system hasn't implemented ." + "The should all be of type map, but it's ." ] - } - }, - "sqlState" : "KD001" - }, - "FAILED_REGISTER_CLASS_WITH_KRYO" : { - "message" : [ - "Failed to register classes with Kryo." - ], - "sqlState" : "KD000" - }, - "FAILED_RENAME_PATH" : { - "message" : [ - "Failed to rename to as destination already exists." - ], - "sqlState" : "42K04" - }, - "FAILED_RENAME_TEMP_FILE" : { - "message" : [ - "Failed to rename temp file to as FileSystem.rename returned false." - ], - "sqlState" : "58030" - }, - "FAILED_ROW_TO_JSON" : { - "message" : [ - "Failed to convert the row value of the class to the target SQL type in the JSON format." - ], - "sqlState" : "2203G" - }, - "FAILED_TO_CREATE_PLAN_FOR_DIRECT_QUERY" : { - "message" : [ - "Failed to create plan for direct query on files: " - ], - "sqlState" : "58030" - }, - "FAILED_TO_LOAD_ROUTINE" : { - "message" : [ - "Failed to load routine ." - ], - "sqlState" : "38000" - }, - "FAILED_TO_PARSE_TOO_COMPLEX" : { - "message" : [ - "The statement, including potential SQL functions and referenced views, was too complex to parse.", - "To mitigate this error divide the statement into multiple, less complex chunks." - ], - "sqlState" : "54001" - }, - "FEATURE_NOT_ENABLED" : { - "message" : [ - "The feature is not enabled. Consider setting the config to to enable this capability." - ], - "sqlState" : "56038" - }, - "FIELD_ALREADY_EXISTS" : { - "message" : [ - "Cannot column, because already exists in ." - ], - "sqlState" : "42710" - }, - "FIELD_NOT_FOUND" : { - "message" : [ - "No such struct field in ." - ], - "sqlState" : "42704" - }, - "FLATMAPGROUPSWITHSTATE_USER_FUNCTION_ERROR" : { - "message" : [ - "An error occurred in the user provided function in flatMapGroupsWithState. Reason: " - ], - "sqlState" : "39000" + }, + "MAP_FUNCTION_DIFF_TYPES" : { + "message" : [ + "Input to should have been followed by a value with same key type, but it's [, ]." + ] + }, + "MAP_ZIP_WITH_DIFF_TYPES" : { + "message" : [ + "Input to the should have been two maps with compatible key types, but it's [, ]." + ] + }, + "NON_FOLDABLE_INPUT" : { + "message" : [ + "the input should be a foldable expression; however, got ." + ] + }, + "NON_STRING_TYPE" : { + "message" : [ + "all arguments of the function must be strings." + ] + }, + "NON_STRUCT_TYPE" : { + "message" : [ + "the input should be a struct expression; however, got ." + ] + }, + "NULL_TYPE" : { + "message" : [ + "Null typed values cannot be used as arguments of ." + ] + }, + "PARAMETER_CONSTRAINT_VIOLATION" : { + "message" : [ + "The () must be the ()." + ] + }, + "RANGE_FRAME_INVALID_TYPE" : { + "message" : [ + "The data type used in the order specification does not support the data type which is used in the range frame." + ] + }, + "RANGE_FRAME_MULTI_ORDER" : { + "message" : [ + "A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: ." + ] + }, + "RANGE_FRAME_WITHOUT_ORDER" : { + "message" : [ + "A range window frame cannot be used in an unordered window specification." + ] + }, + "SEQUENCE_WRONG_INPUT_TYPES" : { + "message" : [ + " uses the wrong parameter type. The parameter type must conform to:", + "1. The start and stop expressions must resolve to the same type.", + "2. If start and stop expressions resolve to the type, then the step expression must resolve to the type.", + "3. Otherwise, if start and stop expressions resolve to the type, then the step expression must resolve to the same type." + ] + }, + "SPECIFIED_WINDOW_FRAME_DIFF_TYPES" : { + "message" : [ + "Window frame bounds and do not have the same type: <> ." + ] + }, + "SPECIFIED_WINDOW_FRAME_INVALID_BOUND" : { + "message" : [ + "Window frame upper bound does not follow the lower bound ." + ] + }, + "SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE" : { + "message" : [ + "The data type of the bound does not match the expected data type ." + ] + }, + "SPECIFIED_WINDOW_FRAME_WITHOUT_FOLDABLE" : { + "message" : [ + "Window frame bound is not a literal." + ] + }, + "SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON" : { + "message" : [ + "The lower bound of a window frame must be to the upper bound." + ] + }, + "STACK_COLUMN_DIFF_TYPES" : { + "message" : [ + "The data type of the column () do not have the same type: () <> ()." + ] + }, + "TYPE_CHECK_FAILURE_WITH_HINT" : { + "message" : [ + "." + ] + }, + "UNEXPECTED_CLASS_TYPE" : { + "message" : [ + "class not found." + ] + }, + "UNEXPECTED_INPUT_TYPE" : { + "message" : [ + "The parameter requires the type, however has the type ." + ] + }, + "UNEXPECTED_NULL" : { + "message" : [ + "The must not be null." + ] + }, + "UNEXPECTED_RETURN_TYPE" : { + "message" : [ + "The requires return type, but the actual is type." + ] + }, + "UNEXPECTED_STATIC_METHOD" : { + "message" : [ + "cannot find a static method that matches the argument types in ." + ] + }, + "UNSUPPORTED_INPUT_TYPE" : { + "message" : [ + "The input of can't be type data." + ] + }, + "VALUE_OUT_OF_RANGE" : { + "message" : [ + "The must be between (current value = )." + ] + }, + "WRONG_NUM_ARG_TYPES" : { + "message" : [ + "The expression requires argument types but the actual number is ." + ] + }, + "WRONG_NUM_ENDPOINTS" : { + "message" : [ + "The number of endpoints must be >= 2 to construct intervals but the actual number is ." + ] + } + }, + "sqlState" : "42K09" }, - "FORBIDDEN_OPERATION" : { + "DATATYPE_MISSING_SIZE" : { "message" : [ - "The operation is not allowed on the : ." + "DataType requires a length parameter, for example (10). Please specify the length." ], - "sqlState" : "42809" + "sqlState" : "42K01" }, - "FOREACH_BATCH_USER_FUNCTION_ERROR" : { + "DATA_SOURCE_ALREADY_EXISTS" : { "message" : [ - "An error occurred in the user provided function in foreach batch sink. Reason: " + "Data source '' already exists. Please choose a different name for the new data source." ], - "sqlState" : "39000" + "sqlState" : "42710" }, - "FOREACH_USER_FUNCTION_ERROR" : { + "DATA_SOURCE_EXTERNAL_ERROR" : { "message" : [ - "An error occurred in the user provided function in foreach sink. Reason: " + "Encountered error when saving to external data source." ], - "sqlState" : "39000" + "sqlState" : "KD010" }, - "GENERATED_COLUMN_WITH_DEFAULT_VALUE" : { + "DATA_SOURCE_NOT_EXIST" : { "message" : [ - "A column cannot have both a default value and a generation expression but column has default value: () and generation expression: ()." + "Data source '' not found. Please make sure the data source is registered." ], - "sqlState" : "42623" + "sqlState" : "42704" }, - "GEO_ENCODER_SRID_MISMATCH_ERROR" : { + "DATA_SOURCE_NOT_FOUND" : { "message" : [ - "Failed to encode value because provided SRID of a value to encode does not match type SRID: ." + "Failed to find the data source: . Make sure the provider name is correct and the package is properly registered and compatible with your Spark version." ], - "sqlState" : "42K09" + "sqlState" : "42K02" }, - "GET_TABLES_BY_TYPE_UNSUPPORTED_BY_HIVE_VERSION" : { + "DATA_SOURCE_TABLE_SCHEMA_MISMATCH" : { "message" : [ - "Hive 2.2 and lower versions don't support getTablesByType. Please use Hive 2.3 or higher version." + "The schema of the data source table does not match the expected schema. If you are using the DataFrameReader.schema API or creating a table, avoid specifying the schema.", + "Data Source schema: ", + "Expected schema: " ], - "sqlState" : "56038" + "sqlState" : "42K03" }, - "GRAPHITE_SINK_INVALID_PROTOCOL" : { + "DATETIME_FIELD_OUT_OF_BOUNDS" : { "message" : [ - "Invalid Graphite protocol: ." + "." ], - "sqlState" : "KD000" + "subClass" : { + "WITHOUT_SUGGESTION" : { + "message" : [ + "" + ] + }, + "WITH_SUGGESTION" : { + "message" : [ + "If necessary set to \"false\" to bypass this error." + ] + } + }, + "sqlState" : "22023" }, - "GRAPHITE_SINK_PROPERTY_MISSING" : { + "DATETIME_OVERFLOW" : { "message" : [ - "Graphite sink requires '' property." + "Datetime operation overflow: ." ], - "sqlState" : "KD000" + "sqlState" : "22008" }, - "GROUPING_COLUMN_MISMATCH" : { + "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION" : { "message" : [ - "Column of grouping () can't be found in grouping columns ." + "Decimal precision exceeds max precision ." ], - "sqlState" : "42803" + "sqlState" : "22003" }, - "GROUPING_ID_COLUMN_MISMATCH" : { + "DEFAULT_DATABASE_NOT_EXISTS" : { "message" : [ - "Columns of grouping_id () does not match grouping columns ()." + "Default database does not exist, please create it first or change default database to ``." ], - "sqlState" : "42803" + "sqlState" : "42704" }, - "GROUPING_SIZE_LIMIT_EXCEEDED" : { + "DEFAULT_PLACEMENT_INVALID" : { "message" : [ - "Grouping sets size cannot be greater than ." + "A DEFAULT keyword in a MERGE, INSERT, UPDATE, or SET VARIABLE command could not be directly assigned to a target column because it was part of an expression.", + "For example: `UPDATE SET c1 = DEFAULT` is allowed, but `UPDATE T SET c1 = DEFAULT + 1` is not allowed." ], - "sqlState" : "54000" + "sqlState" : "42608" }, - "GROUP_BY_AGGREGATE" : { + "DEFAULT_UNSUPPORTED" : { "message" : [ - "Aggregate functions are not allowed in GROUP BY, but found ." + "Failed to execute command because DEFAULT values are not supported for target data source with table provider: \"\"." ], - "sqlState" : "42903" + "sqlState" : "42623" }, - "GROUP_BY_POS_AGGREGATE" : { + "DEFINE_FLOW_ONCE_OPTION_NOT_SUPPORTED" : { "message" : [ - "GROUP BY refers to an expression that contains an aggregate function. Aggregate functions are not allowed in GROUP BY." + "Defining a one-time flow with the 'once' option is not supported." ], - "sqlState" : "42903" + "sqlState" : "0A000" }, - "GROUP_BY_POS_OUT_OF_RANGE" : { + "DESCRIBE_JSON_NOT_EXTENDED" : { "message" : [ - "GROUP BY position is not in select list (valid range is [1, ])." + "DESCRIBE TABLE ... AS JSON only supported when [EXTENDED|FORMATTED] is specified.", + "For example: DESCRIBE EXTENDED AS JSON is supported but DESCRIBE AS JSON is not." ], - "sqlState" : "42805" + "sqlState" : "0A000" }, - "GROUP_EXPRESSION_TYPE_IS_NOT_ORDERABLE" : { + "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED" : { "message" : [ - "The expression cannot be used as a grouping expression because its data type is not an orderable data type." + "Distinct window functions are not supported: ." ], - "sqlState" : "42822" + "sqlState" : "0A000" }, - "HINT_UNSUPPORTED_FOR_JDBC_DIALECT" : { + "DIVIDE_BY_ZERO" : { "message" : [ - "The option `hint` is not supported for in JDBC data source. Supported dialects are `MySQLDialect`, `OracleDialect` and `DatabricksDialect`." + "Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set to \"false\" to bypass this error." ], - "sqlState" : "42822" + "sqlState" : "22012" }, - "HLL_INVALID_INPUT_SKETCH_BUFFER" : { + "DUPLICATED_CTE_NAMES" : { "message" : [ - "Invalid call to ; only valid HLL sketch buffers are supported as inputs (such as those produced by the `hll_sketch_agg` function)." + "CTE definition can't have duplicate names: ." ], - "sqlState" : "22546" + "sqlState" : "42602" }, - "HLL_INVALID_LG_K" : { + "DUPLICATED_FIELD_NAME_IN_ARROW_STRUCT" : { "message" : [ - "Invalid call to ; the `lgConfigK` value must be between and , inclusive: ." + "Duplicated field names in Arrow Struct are not allowed, got ." ], - "sqlState" : "22546" + "sqlState" : "42713" }, - "HLL_K_MUST_BE_CONSTANT" : { + "DUPLICATED_MAP_KEY" : { "message" : [ - "Invalid call to ; the `K` value must be a constant value, but got a non-constant expression." + "Duplicate map key was found, please check the input data.", + "If you want to remove the duplicated keys, you can set to \"LAST_WIN\" so that the key inserted at last takes precedence." ], - "sqlState" : "42K0E" + "sqlState" : "23505" }, - "HLL_UNION_DIFFERENT_LG_K" : { + "DUPLICATED_METRICS_NAME" : { "message" : [ - "Sketches have different `lgConfigK` values: and . Set the `allowDifferentLgConfigK` parameter to true to call with different `lgConfigK` values." + "The metric name is not unique: . The same name cannot be used for metrics with different results.", + "However multiple instances of metrics with with same result and name are allowed (e.g. self-joins)." ], - "sqlState" : "22000" + "sqlState" : "42710" }, - "HYBRID_ANALYZER_EXCEPTION" : { + "DUPLICATE_ASSIGNMENTS" : { "message" : [ - "An failure occurred when attempting to resolve a query or command with both the legacy fixed-point analyzer as well as the single-pass resolver." + "The columns or variables appear more than once as assignment targets." + ], + "sqlState" : "42701" + }, + "DUPLICATE_CLAUSES" : { + "message" : [ + "Found duplicate clauses: . Please, remove one of them." + ], + "sqlState" : "42614" + }, + "DUPLICATE_CONDITION_IN_SCOPE" : { + "message" : [ + "Found duplicate condition in the scope. Please, remove one of them." + ], + "sqlState" : "42734" + }, + "DUPLICATE_EXCEPTION_HANDLER" : { + "message" : [ + "Found duplicate handlers. Please, remove one of them." ], "subClass" : { - "FIXED_POINT_FAILED_SINGLE_PASS_SUCCEEDED" : { + "CONDITION" : { "message" : [ - "Fixed-point resolution failed, but single-pass resolution succeeded.", - "Single-pass analyzer output:", - "" + "Found duplicate handlers for the same condition ." ] }, - "LOGICAL_PLAN_COMPARISON_MISMATCH" : { + "SQLSTATE" : { "message" : [ - "Outputs of fixed-point and single-pass analyzers do not match.", - "Fixed-point analyzer output:", - "", - "Single-pass analyzer output:", - "" + "Found duplicate handlers for the same SQLSTATE ." + ] + } + }, + "sqlState" : "42734" + }, + "DUPLICATE_FLOW_SQL_CONF" : { + "message" : [ + "Found duplicate sql conf for dataset '': '' is defined by both '' and ''" + ], + "sqlState" : "42710" + }, + "DUPLICATE_KEY" : { + "message" : [ + "Found duplicate keys ." + ], + "sqlState" : "23505" + }, + "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT" : { + "message" : [ + "Call to routine is invalid because it includes multiple argument assignments to the same parameter name ." + ], + "subClass" : { + "BOTH_POSITIONAL_AND_NAMED" : { + "message" : [ + "A positional argument and named argument both referred to the same parameter. Please remove the named argument referring to this parameter." ] }, - "OUTPUT_SCHEMA_COMPARISON_MISMATCH" : { + "DOUBLE_NAMED_ARGUMENT_REFERENCE" : { "message" : [ - "Output schemas of fixed-point and single-pass analyzers do not match.", - "Fixed-point analyzer output schema:", - "", - "Single-pass analyzer output schema:", - "" + "More than one named argument referred to the same parameter. Please assign a value only once." ] } }, - "sqlState" : "XX000" + "sqlState" : "4274K" }, - "IDENTIFIER_TOO_MANY_NAME_PARTS" : { + "DUPLICATE_ROUTINE_PARAMETER_NAMES" : { "message" : [ - " is not a valid identifier as it has more than name parts." + "Found duplicate name(s) in the parameter list of the user-defined routine : ." ], - "sqlState" : "42601" + "sqlState" : "42734" }, - "IDENTITY_COLUMNS_DUPLICATED_SEQUENCE_GENERATOR_OPTION" : { + "DUPLICATE_ROUTINE_RETURNS_COLUMNS" : { "message" : [ - "Duplicated IDENTITY column sequence generator option: ." + "Found duplicate column(s) in the RETURNS clause column list of the user-defined routine : ." ], - "sqlState" : "42601" + "sqlState" : "42711" }, - "IDENTITY_COLUMNS_ILLEGAL_STEP" : { + "DUPLICATE_VARIABLE_NAME_INSIDE_DECLARE" : { "message" : [ - "IDENTITY column step cannot be 0." + "Found duplicate variable in the declare variable list. Please, remove one of them." ], - "sqlState" : "42611" + "sqlState" : "42734" }, - "IDENTITY_COLUMNS_UNSUPPORTED_DATA_TYPE" : { + "DYNAMIC_PARTITION_WRITE_PARTITION_NUM_LIMIT_EXCEEDED" : { "message" : [ - "DataType is not supported for IDENTITY columns." + "Number of dynamic partitions created is , which is more than . To solve this try to set to at least ." ], - "sqlState" : "428H2" + "sqlState" : "54054" }, - "IDENTITY_COLUMN_WITH_DEFAULT_VALUE" : { + "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : { "message" : [ - "A column cannot have both a default value and an identity column specification but column has default value: () and identity column specification: ()." + "Previous node emitted a row with eventTime= which is older than current_watermark_value=", + "This can lead to correctness issues in the stateful operators downstream in the execution pipeline.", + "Please correct the operator logic to emit rows after current global watermark value." ], - "sqlState" : "42623" + "sqlState" : "42815" }, - "ILLEGAL_DAY_OF_WEEK" : { + "EMPTY_JSON_FIELD_VALUE" : { "message" : [ - "Illegal input for day of week: ." + "Failed to parse an empty string for data type ." ], - "sqlState" : "22009" + "sqlState" : "42604" }, - "ILLEGAL_STATE_STORE_VALUE" : { + "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE" : { "message" : [ - "Illegal value provided to the State Store" + "The datasource does not support writing empty or nested empty schemas. Please make sure the data schema has at least one or more column(s)." ], - "subClass" : { - "EMPTY_LIST_VALUE" : { - "message" : [ - "Cannot write empty list values to State Store for StateName ." - ] - }, - "NULL_VALUE" : { - "message" : [ - "Cannot write null values to State Store for StateName ." - ] - } - }, - "sqlState" : "42601" + "sqlState" : "0A000" }, - "INCOMPARABLE_PIVOT_COLUMN" : { + "ENCODER_NOT_FOUND" : { "message" : [ - "Invalid pivot column . Pivot columns must be comparable." + "Not found an encoder of the type to Spark SQL internal representation.", + "Consider to change the input type to one of supported at '/sql-ref-datatypes.html'." ], - "sqlState" : "42818" + "sqlState" : "42704" }, - "INCOMPATIBLE_BATCH_VIEW_READ" : { + "END_LABEL_WITHOUT_BEGIN_LABEL" : { "message" : [ - "View is a batch view and must be referenced using SparkSession#read. This check can be disabled by setting Spark conf pipelines.incompatibleViewCheck.enabled = false." + "End label can not exist without begin label." ], - "sqlState" : "42000" + "sqlState" : "42K0L" }, - "INCOMPATIBLE_COLUMN_CHANGES_AFTER_VIEW_WITH_PLAN_CREATION" : { + "ERROR_READING_AVRO_UNKNOWN_FINGERPRINT" : { "message" : [ - "View plan references table whose columns changed since the view plan was initially captured.", - "Column changes:", - "", - "This indicates the table has evolved and the view based on the plan must be recreated." + "Error reading avro data -- encountered an unknown fingerprint: , not sure what schema to use.", + "This could happen if you registered additional schemas after starting your spark context." ], - "sqlState" : "51024" + "sqlState" : "KD00B" }, - "INCOMPATIBLE_COLUMN_TYPE" : { + "EVENT_TIME_IS_NOT_ON_TIMESTAMP_TYPE" : { "message" : [ - " can only be performed on tables with compatible column types. The column of the table is type which is not compatible with at the same column of the first table.." + "The event time has the invalid type , but expected \"TIMESTAMP\"." ], - "sqlState" : "42825" + "sqlState" : "42K09" }, - "INCOMPATIBLE_DATASOURCE_REGISTER" : { + "EVENT_TIME_MUST_BE_TOP_LEVEL_COLUMN" : { "message" : [ - "Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: " + "The event time in withWatermark must be a top-level column, but '' is a nested field. To use it, alias it to a top-level column in a select before withWatermark." ], - "sqlState" : "56038" + "sqlState" : "42K09" }, - "INCOMPATIBLE_DATA_FOR_TABLE" : { + "EXCEED_LIMIT_LENGTH" : { "message" : [ - "Cannot write incompatible data for the table :" + "Exceeds char/varchar type length limitation: ." + ], + "sqlState" : "54006" + }, + "EXCEPT_NESTED_COLUMN_INVALID_TYPE" : { + "message" : [ + "EXCEPT column was resolved and expected to be StructType, but found type ." + ], + "sqlState" : "428H2" + }, + "EXCEPT_OVERLAPPING_COLUMNS" : { + "message" : [ + "Columns in an EXCEPT list must be distinct and non-overlapping, but got ()." + ], + "sqlState" : "42702" + }, + "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES" : { + "message" : [ + "The USING clause of this EXECUTE IMMEDIATE command contained multiple arguments with same alias (), which is invalid; please update the command to specify unique aliases and then try it again." + ], + "sqlState" : "42701" + }, + "EXPECT_PERMANENT_VIEW_NOT_TEMP" : { + "message" : [ + "'' expects a permanent view but is a temp view." + ], + "sqlState" : "42809" + }, + "EXPECT_TABLE_NOT_VIEW" : { + "message" : [ + "'' expects a table but is a view." ], "subClass" : { - "AMBIGUOUS_COLUMN_NAME" : { - "message" : [ - "Ambiguous column name in the input data ." - ] - }, - "CANNOT_FIND_DATA" : { + "NO_ALTERNATIVE" : { "message" : [ - "Cannot find data for the output column ." + "" ] }, - "CANNOT_SAFELY_CAST" : { + "USE_ALTER_VIEW" : { "message" : [ - "Cannot safely cast to ." + "Please use ALTER VIEW instead." ] - }, - "EXTRA_COLUMNS" : { + } + }, + "sqlState" : "42809" + }, + "EXPECT_VIEW_NOT_TABLE" : { + "message" : [ + "The table does not support ." + ], + "subClass" : { + "NO_ALTERNATIVE" : { "message" : [ - "Cannot write extra columns ." + "" ] }, - "EXTRA_STRUCT_FIELDS" : { + "USE_ALTER_TABLE" : { "message" : [ - "Cannot write extra fields to the struct ." + "Please use ALTER TABLE instead." ] - }, - "NULLABLE_ARRAY_ELEMENTS" : { - "message" : [ - "Cannot write nullable elements to array of non-nulls: ." + } + }, + "sqlState" : "42809" + }, + "EXPRESSION_DECODING_FAILED" : { + "message" : [ + "Failed to decode a row to a value of the expressions: ." + ], + "sqlState" : "42846" + }, + "EXPRESSION_ENCODING_FAILED" : { + "message" : [ + "Failed to encode a value of the expressions: to a row." + ], + "sqlState" : "42846" + }, + "EXPRESSION_TRANSLATION_TO_V2_IS_NOT_SUPPORTED" : { + "message" : [ + "Expression cannot be translated to v2 expression." + ], + "sqlState" : "0A000" + }, + "EXPRESSION_TYPE_IS_NOT_ORDERABLE" : { + "message" : [ + "Column expression cannot be sorted because its type is not orderable." + ], + "sqlState" : "42822" + }, + "FAILED_EXECUTE_UDF" : { + "message" : [ + "User defined function (: () => ) failed due to: ." + ], + "sqlState" : "39000" + }, + "FAILED_FUNCTION_CALL" : { + "message" : [ + "Failed preparing of the function for call. Please, double check function's arguments." + ], + "sqlState" : "38000" + }, + "FAILED_JDBC" : { + "message" : [ + "Failed JDBC on the operation:" + ], + "subClass" : { + "ALTER_TABLE" : { + "message" : [ + "Alter the table ." ] }, - "NULLABLE_COLUMN" : { + "CONNECTION" : { "message" : [ - "Cannot write nullable values to non-null column ." + "Couldn't connect to the database" ] }, - "NULLABLE_MAP_VALUES" : { + "CREATE_INDEX" : { "message" : [ - "Cannot write nullable values to map of non-nulls: ." + "Create the index in the table." ] }, - "STRUCT_MISSING_FIELDS" : { + "CREATE_NAMESPACE" : { "message" : [ - "Struct missing fields: ." + "Create the namespace ." ] }, - "UNEXPECTED_COLUMN_NAME" : { + "CREATE_NAMESPACE_COMMENT" : { "message" : [ - "Struct -th field name does not match (may be out of order): expected , found ." + "Create a comment on the namespace: ." ] - } - }, - "sqlState" : "KD000" - }, - "INCOMPATIBLE_JOIN_TYPES" : { - "message" : [ - "The join types and are incompatible." - ], - "sqlState" : "42613" - }, - "INCOMPATIBLE_STREAMING_VIEW_READ" : { - "message" : [ - "View is a streaming view and must be referenced using SparkSession#readStream. This check can be disabled by setting Spark conf pipelines.incompatibleViewCheck.enabled = false." - ], - "sqlState" : "42000" - }, - "INCOMPATIBLE_TABLE_CHANGE_AFTER_ANALYSIS" : { - "message" : [ - "Detected incompatible changes to table after DataFrame/Dataset has been resolved and analyzed, meaning the underlying plan is out of sync. Please, re-create DataFrame/Dataset before attempting to execute the query again." - ], - "subClass" : { - "COLUMNS_MISMATCH" : { + }, + "CREATE_TABLE" : { "message" : [ - "Data columns have changed:", - "" + "Create the table ." ] }, - "METADATA_COLUMNS_MISMATCH" : { + "DROP_INDEX" : { "message" : [ - "Metadata columns have changed:", - "" + "Drop the index in the table." ] }, - "TABLE_ID_MISMATCH" : { + "DROP_NAMESPACE" : { "message" : [ - "Table ID has changed from to ." + "Drop the namespace ." ] - } - }, - "sqlState" : "51024" - }, - "INCOMPATIBLE_VIEW_SCHEMA_CHANGE" : { - "message" : [ - "The SQL query of view has an incompatible schema change and column cannot be resolved. Expected columns named but got .", - "Please try to re-create the view by running: ." - ], - "sqlState" : "51024" - }, - "INCOMPLETE_TYPE_DEFINITION" : { - "message" : [ - "Incomplete complex type:" - ], - "subClass" : { - "ARRAY" : { + }, + "DROP_TABLE" : { "message" : [ - "The definition of \"ARRAY\" type is incomplete. You must provide an element type. For example: \"ARRAY\"." + "Drop the table ." ] }, - "MAP" : { + "GET_TABLES" : { "message" : [ - "The definition of \"MAP\" type is incomplete. You must provide a key type and a value type. For example: \"MAP\"." + "Get tables from the namespace: ." ] }, - "STRUCT" : { + "LIST_NAMESPACES" : { "message" : [ - "The definition of \"STRUCT\" type is incomplete. You must provide at least one field type. For example: \"STRUCT\"." + "List namespaces." + ] + }, + "LOAD_TABLE" : { + "message" : [ + "Load the table ." + ] + }, + "NAMESPACE_EXISTS" : { + "message" : [ + "Check that the namespace exists." + ] + }, + "REMOVE_NAMESPACE_COMMENT" : { + "message" : [ + "Remove a comment on the namespace: ." + ] + }, + "RENAME_TABLE" : { + "message" : [ + "Rename the table to ." + ] + }, + "TABLE_EXISTS" : { + "message" : [ + "Check that the table exists." + ] + }, + "UNCLASSIFIED" : { + "message" : [ + "" ] } }, - "sqlState" : "42K01" + "sqlState" : "HV000" }, - "INCONSISTENT_BEHAVIOR_CROSS_VERSION" : { + "FAILED_PARSE_STRUCT_TYPE" : { "message" : [ - "You may get a different result due to the upgrading to" + "Failed parsing struct: ." + ], + "sqlState" : "22018" + }, + "FAILED_READ_FILE" : { + "message" : [ + "Encountered error while reading file ." ], "subClass" : { - "DATETIME_PATTERN_RECOGNITION" : { + "CANNOT_READ_FILE_FOOTER" : { "message" : [ - "Spark >= 3.0:", - "Fail to recognize pattern in the DateTimeFormatter.", - "1) You can set to \"LEGACY\" to restore the behavior before Spark 3.0.", - "2) You can form a valid datetime pattern with the guide from '/sql-ref-datetime-pattern.html'." + "Could not read footer. Please ensure that the file is in either ORC or Parquet format.", + "If not, please convert it to a valid format. If the file is in the valid format, please check if it is corrupt.", + "If it is, you can choose to either ignore it or fix the corruption." ] }, - "DATETIME_WEEK_BASED_PATTERN" : { + "FILE_NOT_EXIST" : { "message" : [ - "Spark >= 3.0:", - "All week-based patterns are unsupported since Spark 3.0, detected week-based character: .", - "Please use the SQL function EXTRACT instead." + "File does not exist. It is possible the underlying files have been updated.", + "You can explicitly invalidate the cache in Spark by running 'REFRESH TABLE tableName' command in SQL or by recreating the Dataset/DataFrame involved." ] }, - "PARSE_DATETIME_BY_NEW_PARSER" : { + "NO_HINT" : { "message" : [ - "Spark >= 3.0:", - "Fail to parse in the new parser.", - "You can set to \"LEGACY\" to restore the behavior before Spark 3.0, or set to \"CORRECTED\" and treat it as an invalid datetime string." + "" ] }, - "READ_ANCIENT_DATETIME" : { + "PARQUET_COLUMN_DATA_TYPE_MISMATCH" : { "message" : [ - "Spark >= 3.0: reading dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z from files can be ambiguous, as the files may be written by", - "Spark 2.x or legacy versions of Hive, which uses a legacy hybrid calendar that is different from Spark 3.0+'s Proleptic Gregorian calendar.", - "See more details in SPARK-31404.", - "You can set the SQL config or the datasource option