diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 3e1655d80e4e7..943c2c12cd078 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3393,5 +3393,130 @@ "message" : [ "Write is not supported for binary file data source" ] + }, + "_LEGACY_ERROR_TEMP_2076" : { + "message" : [ + "The length of is , which exceeds the max length allowed: ." + ] + }, + "_LEGACY_ERROR_TEMP_2077" : { + "message" : [ + "Unsupported field name: " + ] + }, + "_LEGACY_ERROR_TEMP_2078" : { + "message" : [ + "Both '' and '' can not be specified at the same time." + ] + }, + "_LEGACY_ERROR_TEMP_2079" : { + "message" : [ + "Option '' or '' is required." + ] + }, + "_LEGACY_ERROR_TEMP_2080" : { + "message" : [ + "Option `` can not be empty." + ] + }, + "_LEGACY_ERROR_TEMP_2081" : { + "message" : [ + "Invalid value `` for parameter ``. This can be `NONE`, `READ_UNCOMMITTED`, `READ_COMMITTED`, `REPEATABLE_READ` or `SERIALIZABLE`." + ] + }, + "_LEGACY_ERROR_TEMP_2082" : { + "message" : [ + "Can't get JDBC type for " + ] + }, + "_LEGACY_ERROR_TEMP_2083" : { + "message" : [ + "Unsupported type " + ] + }, + "_LEGACY_ERROR_TEMP_2084" : { + "message" : [ + "Unsupported array element type based on binary" + ] + }, + "_LEGACY_ERROR_TEMP_2085" : { + "message" : [ + "Nested arrays unsupported" + ] + }, + "_LEGACY_ERROR_TEMP_2086" : { + "message" : [ + "Can't translate non-null value for field " + ] + }, + "_LEGACY_ERROR_TEMP_2087" : { + "message" : [ + "Invalid value `` for parameter `` in table writing via JDBC. The minimum value is 1." + ] + }, + "_LEGACY_ERROR_TEMP_2088" : { + "message" : [ + " is not supported yet." + ] + }, + "_LEGACY_ERROR_TEMP_2089" : { + "message" : [ + "DataType: " + ] + }, + "_LEGACY_ERROR_TEMP_2090" : { + "message" : [ + "The input filter of should be fully convertible." + ] + }, + "_LEGACY_ERROR_TEMP_2091" : { + "message" : [ + "Could not read footer for file: " + ] + }, + "_LEGACY_ERROR_TEMP_2092" : { + "message" : [ + "Could not read footer for file: " + ] + }, + "_LEGACY_ERROR_TEMP_2093" : { + "message" : [ + "Found duplicate field(s) \"\": in case-insensitive mode" + ] + }, + "_LEGACY_ERROR_TEMP_2094" : { + "message" : [ + "Found duplicate field(s) \"\": in id mapping mode" + ] + }, + "_LEGACY_ERROR_TEMP_2095" : { + "message" : [ + "Failed to merge incompatible schemas and " + ] + }, + "_LEGACY_ERROR_TEMP_2096" : { + "message" : [ + " is not supported temporarily." + ] + }, + "_LEGACY_ERROR_TEMP_2097" : { + "message" : [ + "Could not execute broadcast in secs. You can increase the timeout for broadcasts via or disable broadcast join by setting to -1" + ] + }, + "_LEGACY_ERROR_TEMP_2098" : { + "message" : [ + "Could not compare cost with " + ] + }, + "_LEGACY_ERROR_TEMP_2099" : { + "message" : [ + "Unsupported data type:
" + ] + }, + "_LEGACY_ERROR_TEMP_2100" : { + "message" : [ + "not support type: " + ] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index a3e1b980d1fa7..9b043957d2cf7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.errors import java.io.{FileNotFoundException, IOException} import java.lang.reflect.InvocationTargetException import java.net.{URISyntaxException, URL} -import java.sql.{SQLException, SQLFeatureNotSupportedException} +import java.sql.{SQLFeatureNotSupportedException} import java.time.{DateTimeException, LocalDate} import java.time.temporal.ChronoField import java.util.ConcurrentModificationException @@ -967,39 +967,55 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def fileLengthExceedsMaxLengthError(status: FileStatus, maxLength: Int): Throwable = { new SparkException( - s"The length of ${status.getPath} is ${status.getLen}, " + - s"which exceeds the max length allowed: ${maxLength}.") + errorClass = "_LEGACY_ERROR_TEMP_2076", + messageParameters = Map( + "path" -> status.getPath.toString(), + "len" -> status.getLen.toString(), + "maxLength" -> maxLength.toString()), + cause = null) } - def unsupportedFieldNameError(fieldName: String): Throwable = { - new RuntimeException(s"Unsupported field name: ${fieldName}") + def unsupportedFieldNameError(fieldName: String): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2077", + messageParameters = Map("fieldName" -> fieldName)) } def cannotSpecifyBothJdbcTableNameAndQueryError( - jdbcTableName: String, jdbcQueryString: String): Throwable = { - new IllegalArgumentException( - s"Both '$jdbcTableName' and '$jdbcQueryString' can not be specified at the same time.") + jdbcTableName: String, jdbcQueryString: String): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2078", + messageParameters = Map( + "jdbcTableName" -> jdbcTableName, + "jdbcQueryString" -> jdbcQueryString)) } def missingJdbcTableNameAndQueryError( - jdbcTableName: String, jdbcQueryString: String): Throwable = { - new IllegalArgumentException( - s"Option '$jdbcTableName' or '$jdbcQueryString' is required." - ) + jdbcTableName: String, jdbcQueryString: String): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2079", + messageParameters = Map( + "jdbcTableName" -> jdbcTableName, + "jdbcQueryString" -> jdbcQueryString)) } - def emptyOptionError(optionName: String): Throwable = { - new IllegalArgumentException(s"Option `$optionName` can not be empty.") + def emptyOptionError(optionName: String): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2080", + messageParameters = Map("optionName" -> optionName)) } - def invalidJdbcTxnIsolationLevelError(jdbcTxnIsolationLevel: String, value: String): Throwable = { - new IllegalArgumentException( - s"Invalid value `$value` for parameter `$jdbcTxnIsolationLevel`. This can be " + - "`NONE`, `READ_UNCOMMITTED`, `READ_COMMITTED`, `REPEATABLE_READ` or `SERIALIZABLE`.") + def invalidJdbcTxnIsolationLevelError( + jdbcTxnIsolationLevel: String, value: String): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2081", + messageParameters = Map("value" -> value, "jdbcTxnIsolationLevel" -> jdbcTxnIsolationLevel)) } - def cannotGetJdbcTypeError(dt: DataType): Throwable = { - new IllegalArgumentException(s"Can't get JDBC type for ${dt.catalogString}") + def cannotGetJdbcTypeError(dt: DataType): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2082", + messageParameters = Map("catalogString" -> dt.catalogString)) } def unrecognizedSqlTypeError(sqlType: Int): Throwable = { @@ -1008,27 +1024,35 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("typeName" -> sqlType.toString)) } - def unsupportedJdbcTypeError(content: String): Throwable = { - new SQLException(s"Unsupported type $content") + def unsupportedJdbcTypeError(content: String): SparkSQLException = { + new SparkSQLException( + errorClass = "_LEGACY_ERROR_TEMP_2083", + messageParameters = Map("content" -> content)) } - def unsupportedArrayElementTypeBasedOnBinaryError(dt: DataType): Throwable = { - new IllegalArgumentException(s"Unsupported array element " + - s"type ${dt.catalogString} based on binary") + def unsupportedArrayElementTypeBasedOnBinaryError(dt: DataType): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2084", + messageParameters = Map("catalogString" -> dt.catalogString)) } - def nestedArraysUnsupportedError(): Throwable = { - new IllegalArgumentException("Nested arrays unsupported") + def nestedArraysUnsupportedError(): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2085", + messageParameters = Map.empty) } - def cannotTranslateNonNullValueForFieldError(pos: Int): Throwable = { - new IllegalArgumentException(s"Can't translate non-null value for field $pos") + def cannotTranslateNonNullValueForFieldError(pos: Int): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2086", + messageParameters = Map("pos" -> pos.toString())) } - def invalidJdbcNumPartitionsError(n: Int, jdbcNumPartitions: String): Throwable = { - new IllegalArgumentException( - s"Invalid value `$n` for parameter `$jdbcNumPartitions` in table writing " + - "via JDBC. The minimum value is 1.") + def invalidJdbcNumPartitionsError( + n: Int, jdbcNumPartitions: String): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2087", + messageParameters = Map("n" -> n.toString(), "jdbcNumPartitions" -> jdbcNumPartitions)) } def transactionUnsupportedByJdbcServerError(): Throwable = { @@ -1037,72 +1061,99 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map.empty[String, String]) } - def dataTypeUnsupportedYetError(dataType: DataType): Throwable = { - new UnsupportedOperationException(s"$dataType is not supported yet.") + def dataTypeUnsupportedYetError(dataType: DataType): SparkUnsupportedOperationException = { + new SparkUnsupportedOperationException( + errorClass = "_LEGACY_ERROR_TEMP_2088", + messageParameters = Map("dataType" -> dataType.toString())) } - def unsupportedOperationForDataTypeError(dataType: DataType): Throwable = { - new UnsupportedOperationException(s"DataType: ${dataType.catalogString}") + def unsupportedOperationForDataTypeError( + dataType: DataType): SparkUnsupportedOperationException = { + new SparkUnsupportedOperationException( + errorClass = "_LEGACY_ERROR_TEMP_2089", + messageParameters = Map("catalogString" -> dataType.catalogString)) } def inputFilterNotFullyConvertibleError(owner: String): Throwable = { - new SparkException(s"The input filter of $owner should be fully convertible.") + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2090", + messageParameters = Map("owner" -> owner), + cause = null) } def cannotReadFooterForFileError(file: Path, e: IOException): Throwable = { - new SparkException(s"Could not read footer for file: $file", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2091", + messageParameters = Map("file" -> file.toString()), + cause = e) } def cannotReadFooterForFileError(file: FileStatus, e: RuntimeException): Throwable = { - new IOException(s"Could not read footer for file: $file", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2092", + messageParameters = Map("file" -> file.toString()), + cause = e) } def foundDuplicateFieldInCaseInsensitiveModeError( - requiredFieldName: String, matchedOrcFields: String): Throwable = { - new RuntimeException( - s""" - |Found duplicate field(s) "$requiredFieldName": $matchedOrcFields - |in case-insensitive mode - """.stripMargin.replaceAll("\n", " ")) + requiredFieldName: String, matchedOrcFields: String): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2093", + messageParameters = Map( + "requiredFieldName" -> requiredFieldName, + "matchedOrcFields" -> matchedOrcFields)) } def foundDuplicateFieldInFieldIdLookupModeError( - requiredId: Int, matchedFields: String): Throwable = { - new RuntimeException( - s""" - |Found duplicate field(s) "$requiredId": $matchedFields - |in id mapping mode - """.stripMargin.replaceAll("\n", " ")) + requiredId: Int, matchedFields: String): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2094", + messageParameters = Map( + "requiredId" -> requiredId.toString(), + "matchedFields" -> matchedFields)) } def failedToMergeIncompatibleSchemasError( left: StructType, right: StructType, e: Throwable): Throwable = { - new SparkException(s"Failed to merge incompatible schemas $left and $right", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2095", + messageParameters = Map("left" -> left.toString(), "right" -> right.toString()), + cause = e) } - def ddlUnsupportedTemporarilyError(ddl: String): Throwable = { - new UnsupportedOperationException(s"$ddl is not supported temporarily.") + def ddlUnsupportedTemporarilyError(ddl: String): SparkUnsupportedOperationException = { + new SparkUnsupportedOperationException( + errorClass = "_LEGACY_ERROR_TEMP_2096", + messageParameters = Map("ddl" -> ddl)) } def executeBroadcastTimeoutError(timeout: Long, ex: Option[TimeoutException]): Throwable = { new SparkException( - s""" - |Could not execute broadcast in $timeout secs. You can increase the timeout - |for broadcasts via ${SQLConf.BROADCAST_TIMEOUT.key} or disable broadcast join - |by setting ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1 - """.stripMargin.replaceAll("\n", " "), ex.orNull) + errorClass = "_LEGACY_ERROR_TEMP_2097", + messageParameters = Map( + "timeout" -> timeout.toString(), + "broadcastTimeout" -> toSQLConf(SQLConf.BROADCAST_TIMEOUT.key), + "autoBroadcastJoinThreshold" -> toSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key)), + cause = ex.orNull) } - def cannotCompareCostWithTargetCostError(cost: String): Throwable = { - new IllegalArgumentException(s"Could not compare cost with $cost") + def cannotCompareCostWithTargetCostError(cost: String): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2098", + messageParameters = Map("cost" -> cost)) } - def unsupportedDataTypeError(dt: String): Throwable = { - new UnsupportedOperationException(s"Unsupported data type: ${dt}") + def unsupportedDataTypeError(dt: String): SparkUnsupportedOperationException = { + new SparkUnsupportedOperationException( + errorClass = "_LEGACY_ERROR_TEMP_2099", + messageParameters = Map("dt" -> dt)) } def notSupportTypeError(dataType: DataType): Throwable = { - new Exception(s"not support type: $dataType") + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2100", + messageParameters = Map("dataType" -> dataType.toString()), + cause = null) } def notSupportNonPrimitiveTypeError(): Throwable = {