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
125 changes: 125 additions & 0 deletions core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -3393,5 +3393,130 @@
"message" : [
"Write is not supported for binary file data source"
]
},
"_LEGACY_ERROR_TEMP_2076" : {
"message" : [
"The length of <path> is <len>, which exceeds the max length allowed: <maxLength>."
]
},
"_LEGACY_ERROR_TEMP_2077" : {
"message" : [
"Unsupported field name: <fieldName>"
]
},
"_LEGACY_ERROR_TEMP_2078" : {
"message" : [
"Both '<jdbcTableName>' and '<jdbcQueryString>' can not be specified at the same time."
]
},
"_LEGACY_ERROR_TEMP_2079" : {
"message" : [
"Option '<jdbcTableName>' or '<jdbcQueryString>' is required."
]
},
"_LEGACY_ERROR_TEMP_2080" : {
"message" : [
"Option `<optionName>` can not be empty."
]
},
"_LEGACY_ERROR_TEMP_2081" : {
"message" : [
"Invalid value `<value>` for parameter `<jdbcTxnIsolationLevel>`. This can be `NONE`, `READ_UNCOMMITTED`, `READ_COMMITTED`, `REPEATABLE_READ` or `SERIALIZABLE`."
]
},
"_LEGACY_ERROR_TEMP_2082" : {
"message" : [
"Can't get JDBC type for <catalogString>"
]
},
"_LEGACY_ERROR_TEMP_2083" : {
"message" : [
"Unsupported type <content>"
]
},
"_LEGACY_ERROR_TEMP_2084" : {
"message" : [
"Unsupported array element type <catalogString> based on binary"
]
},
"_LEGACY_ERROR_TEMP_2085" : {
"message" : [
"Nested arrays unsupported"
]
},
"_LEGACY_ERROR_TEMP_2086" : {
"message" : [
"Can't translate non-null value for field <pos>"
]
},
"_LEGACY_ERROR_TEMP_2087" : {
"message" : [
"Invalid value `<n>` for parameter `<jdbcNumPartitions>` in table writing via JDBC. The minimum value is 1."
]
},
"_LEGACY_ERROR_TEMP_2088" : {
"message" : [
"<dataType> is not supported yet."
]
},
"_LEGACY_ERROR_TEMP_2089" : {
"message" : [
"DataType: <catalogString>"
]
},
"_LEGACY_ERROR_TEMP_2090" : {
"message" : [
"The input filter of <owner> should be fully convertible."
]
},
"_LEGACY_ERROR_TEMP_2091" : {
"message" : [
"Could not read footer for file: <file>"
]
},
"_LEGACY_ERROR_TEMP_2092" : {
"message" : [
"Could not read footer for file: <file>"
]
},
"_LEGACY_ERROR_TEMP_2093" : {
"message" : [
"Found duplicate field(s) \"<requiredFieldName>\": <matchedOrcFields> in case-insensitive mode"
]
},
"_LEGACY_ERROR_TEMP_2094" : {
"message" : [
"Found duplicate field(s) \"<requiredId>\": <matchedFields> in id mapping mode"
]
},
"_LEGACY_ERROR_TEMP_2095" : {
"message" : [
"Failed to merge incompatible schemas <left> and <right>"
]
},
"_LEGACY_ERROR_TEMP_2096" : {
"message" : [
"<ddl> is not supported temporarily."
]
},
"_LEGACY_ERROR_TEMP_2097" : {
"message" : [
"Could not execute broadcast in <timeout> secs. You can increase the timeout for broadcasts via <broadcastTimeout> or disable broadcast join by setting <autoBroadcastJoinThreshold> to -1"
]
},
"_LEGACY_ERROR_TEMP_2098" : {
"message" : [
"Could not compare cost with <cost>"
]
},
"_LEGACY_ERROR_TEMP_2099" : {
"message" : [
"Unsupported data type: <dt>"
]
},
"_LEGACY_ERROR_TEMP_2100" : {
"message" : [
"not support type: <dataType>"
]
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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 = {
Expand All @@ -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 = {
Expand All @@ -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 = {
Expand Down