Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 30 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -1216,6 +1216,12 @@
],
"sqlState" : "42K01"
},
"DATATYPE_OPERATION_UNSUPPORTED" : {
"message" : [
"The operation `dataType` is not supported."
],
"sqlState" : "0A000"
},
"DATA_SOURCE_ALREADY_EXISTS" : {
"message" : [
"Data source '<provider>' already exists. Please choose a different name for the new data source."
Expand Down Expand Up @@ -1297,6 +1303,12 @@
],
"sqlState" : "42623"
},
"DEPRECATED_METHOD_CALL" : {
"message" : [
"The method `<oldMethod>` is deprecated, please use `<newMethod>` instead."
],
"sqlState" : "01000"
},
"DESCRIBE_JSON_NOT_EXTENDED" : {
"message" : [
"DESCRIBE TABLE ... AS JSON only supported when [EXTENDED|FORMATTED] is specified.",
Expand Down Expand Up @@ -4072,6 +4084,12 @@
],
"sqlState" : "42803"
},
"MISSING_STATIC_PARTITION_COLUMN" : {
"message" : [
"Unknown static partition column: <staticName>."
],
"sqlState" : "42703"
},
"MISSING_TIMEOUT_CONFIGURATION" : {
"message" : [
"The operation has timed out, but no timeout duration is configured. To set a processing time-based timeout, use 'GroupState.setTimeoutDuration()' in your 'mapGroupsWithState' or 'flatMapGroupsWithState' operation. For event-time-based timeout, use 'GroupState.setTimeoutTimestamp()' and define a watermark using 'Dataset.withWatermark()'."
Expand Down Expand Up @@ -4126,6 +4144,12 @@
],
"sqlState" : "0A000"
},
"MUST_OVERRIDE_METHOD" : {
"message" : [
"You must override one `<methodName>`. It's preferred to not override the deprecated one."
],
"sqlState" : "01000"
},
"NAMED_PARAMETERS_NOT_SUPPORTED" : {
"message" : [
"Named parameters are not supported for function <functionName>; please retry the query with positional arguments to the function call instead."
Expand Down Expand Up @@ -5449,6 +5473,12 @@
],
"sqlState" : "4274K"
},
"UNEXPECTED_REQUIRED_PARAMETER" : {
"message" : [
"Routine <routineName> has an unexpected required argument for the provided routine signature <parameters>. All required arguments should come before optional arguments."
],
"sqlState" : "4274K"
},
"UNEXPECTED_SERIALIZER_FOR_CLASS" : {
"message" : [
"The class <className> has an unexpected expression serializer. Expects \"STRUCT\" or \"IF\" which returns \"STRUCT\" but found <expr>."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.util.Locale

import org.apache.hadoop.fs.Path

import org.apache.spark.{SPARK_DOC_ROOT, SparkException, SparkRuntimeException, SparkThrowable, SparkUnsupportedOperationException}
import org.apache.spark.{SPARK_DOC_ROOT, SparkThrowable, SparkUnsupportedOperationException}
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, FunctionIdentifier, InternalRow, QualifiedTableName, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, FunctionAlreadyExistsException, NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchPartitionException, NoSuchTableException, Star, TableAlreadyExistsException, UnresolvedRegex}
Expand Down Expand Up @@ -54,10 +54,11 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat

def unexpectedRequiredParameter(
routineName: String, parameters: Seq[InputParameter]): Throwable = {
val errorMessage = s"Routine ${toSQLId(routineName)} has an unexpected required argument for" +
s" the provided routine signature ${parameters.mkString("[", ", ", "]")}." +
s" All required arguments should come before optional arguments."
SparkException.internalError(errorMessage)
new AnalysisException(
errorClass = "UNEXPECTED_REQUIRED_PARAMETER",
messageParameters = Map(
"routineName" -> toSQLId(routineName),
"parameters" -> parameters.mkString("[", ", ", "]")))
}

def namedArgumentsNotSupported(functionName: String) : Throwable = {
Expand Down Expand Up @@ -323,7 +324,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
}

def missingStaticPartitionColumn(staticName: String): Throwable = {
SparkException.internalError(s"Unknown static partition column: $staticName.")
new AnalysisException(
errorClass = "MISSING_STATIC_PARTITION_COLUMN",
messageParameters = Map("staticName" -> staticName))
}

def staticPartitionInUserSpecifiedColumnsError(staticName: String): Throwable = {
Expand Down Expand Up @@ -3980,8 +3983,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
}

def dataTypeOperationUnsupportedError(): Throwable = {
SparkException.internalError(
"The operation `dataType` is not supported.")
new SparkUnsupportedOperationException(
"DATATYPE_OPERATION_UNSUPPORTED",
Map.empty[String, String])
}

def nullableRowIdError(nullableRowIdAttrs: Seq[AttributeReference]): Throwable = {
Expand Down Expand Up @@ -4191,21 +4195,22 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
}

private def callDeprecatedMethodError(oldMethod: String, newMethod: String): Throwable = {
SparkException.internalError(s"The method `$oldMethod` is deprecated, " +
s"please use `$newMethod` instead.")
new AnalysisException(
errorClass = "DEPRECATED_METHOD_CALL",
messageParameters = Map(
"oldMethod" -> oldMethod,
"newMethod" -> newMethod))
}

def createTableDeprecatedError(): Throwable = {
callDeprecatedMethodError("createTable(..., StructType, ...)",
"createTable(..., Array[Column], ...)")
}

def mustOverrideOneMethodError(methodName: String): RuntimeException = {
val msg = s"You must override one `$methodName`. It's preferred to not override the " +
"deprecated one."
new SparkRuntimeException(
"INTERNAL_ERROR",
Map("message" -> msg))
def mustOverrideOneMethodError(methodName: String): Throwable = {
new AnalysisException(
errorClass = "MUST_OVERRIDE_METHOD",
messageParameters = Map("methodName" -> methodName))
}

def cannotAssignEventTimeColumn(): Throwable = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import org.apache.spark.sql._
import org.apache.spark.sql.api.java.{UDF1, UDF2, UDF23Test}
import org.apache.spark.sql.catalyst.expressions.{Coalesce, Literal, UnsafeRow}
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.plans.logical.InputParameter
import org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand
import org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter
import org.apache.spark.sql.expressions.SparkUserDefinedFunction
Expand Down Expand Up @@ -1106,6 +1107,62 @@ class QueryCompilationErrorsSuite
ExpectedContext(fragment = "aggregate(array(1,2,3), x -> x + 1, 0)", start = 7, stop = 44)
)
}

test("UNEXPECTED_REQUIRED_PARAMETER: routine has unexpected required parameter") {
val parameters = Seq(
InputParameter("param1", Some(Literal("default1"))),
InputParameter("param2", Some(Literal("default2")))
)
checkError(
exception = QueryCompilationErrors.unexpectedRequiredParameter("testRoutine", parameters)
.asInstanceOf[AnalysisException],
condition = "UNEXPECTED_REQUIRED_PARAMETER",
parameters = Map(
"routineName" -> "`testRoutine`",
"parameters" ->
"[InputParameter(param1,Some(default1)), InputParameter(param2,Some(default2))]"
)
)
}

test("MISSING_STATIC_PARTITION_COLUMN: unknown static partition column") {
checkError(
exception = QueryCompilationErrors.missingStaticPartitionColumn("unknownColumn")
.asInstanceOf[AnalysisException],
condition = "MISSING_STATIC_PARTITION_COLUMN",
parameters = Map("staticName" -> "unknownColumn")
)
}

test("DATATYPE_OPERATION_UNSUPPORTED: dataType operation not supported") {
checkError(
exception = QueryCompilationErrors.dataTypeOperationUnsupportedError()
.asInstanceOf[SparkUnsupportedOperationException],
condition = "DATATYPE_OPERATION_UNSUPPORTED",
parameters = Map.empty
)
}

test("DEPRECATED_METHOD_CALL: deprecated method call") {
checkError(
exception = QueryCompilationErrors.createTableDeprecatedError()
.asInstanceOf[AnalysisException],
condition = "DEPRECATED_METHOD_CALL",
parameters = Map(
"oldMethod" -> "createTable(..., StructType, ...)",
"newMethod" -> "createTable(..., Array[Column], ...)"
)
)
}

test("MUST_OVERRIDE_METHOD: must override one method") {
checkError(
exception = QueryCompilationErrors.mustOverrideOneMethodError("testMethod")
.asInstanceOf[AnalysisException],
condition = "MUST_OVERRIDE_METHOD",
parameters = Map("methodName" -> "testMethod")
)
}
}

class MyCastToString extends SparkUserDefinedFunction(
Expand Down