Skip to content

Commit

Permalink
Merge branch 'master' into rastko.sapic@databricks.com/fix-cast-error…
Browse files Browse the repository at this point in the history
…-in-array-column-types
  • Loading branch information
RaleSapic authored Oct 10, 2024
2 parents 8be0dab + e589ccd commit ab3265e
Show file tree
Hide file tree
Showing 140 changed files with 1,554 additions and 1,048 deletions.
4 changes: 4 additions & 0 deletions .github/workflows/build_and_test.yml
Original file line number Diff line number Diff line change
Expand Up @@ -1112,6 +1112,10 @@ jobs:
with:
distribution: zulu
java-version: ${{ inputs.java }}
- name: Install R
run: |
sudo apt update
sudo apt-get install r-base
- name: Start Minikube
uses: medyagh/setup-minikube@v0.0.18
with:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -456,7 +456,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig
val e = intercept[SparkException] {
fetchCollation(collationName)
}
assert(e.getErrorClass === "COLLATION_INVALID_NAME")
assert(e.getCondition === "COLLATION_INVALID_NAME")
assert(e.getMessageParameters.asScala === Map(
"collationName" -> collationName, "proposals" -> proposals))
}
Expand Down
20 changes: 15 additions & 5 deletions common/utils/src/main/java/org/apache/spark/SparkThrowable.java
Original file line number Diff line number Diff line change
Expand Up @@ -35,19 +35,29 @@
*/
@Evolving
public interface SparkThrowable {
// Succinct, human-readable, unique, and consistent representation of the error category
// If null, error class is not set
String getErrorClass();
/**
* Succinct, human-readable, unique, and consistent representation of the error condition.
* If null, error condition is not set.
*/
String getCondition();

/**
* Succinct, human-readable, unique, and consistent representation of the error category.
* If null, error class is not set.
* @deprecated Use {@link #getCondition()} instead.
*/
@Deprecated
default String getErrorClass() { return getCondition(); }

// Portable error identifier across SQL engines
// If null, error class or SQLSTATE is not set
default String getSqlState() {
return SparkThrowableHelper.getSqlState(this.getErrorClass());
return SparkThrowableHelper.getSqlState(this.getCondition());
}

// True if this error is an internal error.
default boolean isInternalError() {
return SparkThrowableHelper.isInternalError(this.getErrorClass());
return SparkThrowableHelper.isInternalError(this.getCondition());
}

default Map<String, String> getMessageParameters() {
Expand Down
43 changes: 23 additions & 20 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -2543,6 +2543,11 @@
"Interval string does not match second-nano format of ss.nnnnnnnnn."
]
},
"TIMEZONE_INTERVAL_OUT_OF_RANGE" : {
"message" : [
"The interval value must be in the range of [-18, +18] hours with second precision."
]
},
"UNKNOWN_PARSING_ERROR" : {
"message" : [
"Unknown error when parsing <word>."
Expand Down Expand Up @@ -3830,6 +3835,12 @@
],
"sqlState" : "42000"
},
"PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" : {
"message" : [
"The expression <expression> must be inside 'partitionedBy'."
],
"sqlState" : "42S23"
},
"PATH_ALREADY_EXISTS" : {
"message" : [
"Path <outputPath> already exists. Set mode as \"overwrite\" to overwrite the existing path."
Expand Down Expand Up @@ -4002,6 +4013,18 @@
],
"sqlState" : "22023"
},
"SCALAR_FUNCTION_NOT_COMPATIBLE" : {
"message" : [
"ScalarFunction <scalarFunc> not overrides method 'produceResult(InternalRow)' with custom implementation."
],
"sqlState" : "42K0O"
},
"SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED" : {
"message" : [
"ScalarFunction <scalarFunc> not implements or overrides method 'produceResult(InternalRow)'."
],
"sqlState" : "42K0P"
},
"SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION" : {
"message" : [
"The correlated scalar subquery '<sqlExpr>' is neither present in GROUP BY, nor in an aggregate function.",
Expand Down Expand Up @@ -5691,11 +5714,6 @@
"Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`."
]
},
"_LEGACY_ERROR_TEMP_0044" : {
"message" : [
"The interval value must be in the range of [-18, +18] hours with second precision."
]
},
"_LEGACY_ERROR_TEMP_0045" : {
"message" : [
"Invalid time zone displacement value."
Expand Down Expand Up @@ -6702,11 +6720,6 @@
"The pivot column <pivotColumn> has more than <maxValues> distinct values, this could indicate an error. If this was intended, set <config> to at least the number of distinct values of the pivot column."
]
},
"_LEGACY_ERROR_TEMP_1325" : {
"message" : [
"Cannot modify the value of a static config: <key>."
]
},
"_LEGACY_ERROR_TEMP_1327" : {
"message" : [
"Command execution is not supported in runner <runner>."
Expand Down Expand Up @@ -7957,11 +7970,6 @@
"<expr> is not currently supported"
]
},
"_LEGACY_ERROR_TEMP_3055" : {
"message" : [
"ScalarFunction <scalarFunc> neither implement magic method nor override 'produceResult'"
]
},
"_LEGACY_ERROR_TEMP_3056" : {
"message" : [
"Unexpected row-level read relations (allow multiple = <allowMultipleReads>): <other>"
Expand Down Expand Up @@ -8320,11 +8328,6 @@
"Partitions truncate is not supported"
]
},
"_LEGACY_ERROR_TEMP_3146" : {
"message" : [
"Cannot find a compatible ScalarFunction#produceResult"
]
},
"_LEGACY_ERROR_TEMP_3147" : {
"message" : [
"<description>: Batch scan are not supported"
Expand Down
18 changes: 18 additions & 0 deletions common/utils/src/main/resources/error/error-states.json
Original file line number Diff line number Diff line change
Expand Up @@ -4631,6 +4631,18 @@
"standard": "N",
"usedBy": ["Spark"]
},
"42K0O": {
"description": "ScalarFunction not overrides method 'produceResult(InternalRow)' with custom implementation.",
"origin": "Spark",
"standard": "N",
"usedBy": ["Spark"]
},
"42K0P": {
"description": "ScalarFunction not implements or overrides method 'produceResult(InternalRow)'.",
"origin": "Spark",
"standard": "N",
"usedBy": ["Spark"]
},
"42KD0": {
"description": "Ambiguous name reference.",
"origin": "Databricks",
Expand Down Expand Up @@ -4901,6 +4913,12 @@
"standard": "N",
"usedBy": ["SQL Server"]
},
"42S23": {
"description": "Partition transform expression not in 'partitionedBy'",
"origin": "Spark",
"standard": "N",
"usedBy": ["Spark"]
},
"44000": {
"description": "with check option violation",
"origin": "SQL/Foundation",
Expand Down
34 changes: 17 additions & 17 deletions common/utils/src/main/scala/org/apache/spark/SparkException.scala
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ class SparkException(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull

override def getQueryContext: Array[QueryContext] = context
}
Expand Down Expand Up @@ -179,7 +179,7 @@ private[spark] class SparkUpgradeException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
}

/**
Expand Down Expand Up @@ -212,7 +212,7 @@ private[spark] class SparkArithmeticException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
override def getQueryContext: Array[QueryContext] = context
}

Expand Down Expand Up @@ -250,7 +250,7 @@ private[spark] class SparkUnsupportedOperationException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
}

private[spark] object SparkUnsupportedOperationException {
Expand Down Expand Up @@ -280,7 +280,7 @@ private[spark] class SparkClassNotFoundException(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass
override def getCondition: String = errorClass
}

/**
Expand All @@ -296,7 +296,7 @@ private[spark] class SparkConcurrentModificationException(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass
override def getCondition: String = errorClass
}

/**
Expand Down Expand Up @@ -346,7 +346,7 @@ private[spark] class SparkDateTimeException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
override def getQueryContext: Array[QueryContext] = context
}

Expand All @@ -362,7 +362,7 @@ private[spark] class SparkFileNotFoundException(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass
override def getCondition: String = errorClass
}

/**
Expand Down Expand Up @@ -396,7 +396,7 @@ private[spark] class SparkNumberFormatException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
override def getQueryContext: Array[QueryContext] = context
}

Expand Down Expand Up @@ -448,7 +448,7 @@ private[spark] class SparkIllegalArgumentException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
override def getQueryContext: Array[QueryContext] = context
}

Expand Down Expand Up @@ -477,7 +477,7 @@ private[spark] class SparkRuntimeException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
override def getQueryContext: Array[QueryContext] = context
}

Expand Down Expand Up @@ -506,7 +506,7 @@ private[spark] class SparkPythonException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
override def getQueryContext: Array[QueryContext] = context
}

Expand All @@ -524,7 +524,7 @@ private[spark] class SparkNoSuchElementException(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass
override def getCondition: String = errorClass

override def getQueryContext: Array[QueryContext] = context
}
Expand All @@ -541,7 +541,7 @@ private[spark] class SparkSecurityException(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass
override def getCondition: String = errorClass
}

/**
Expand Down Expand Up @@ -575,7 +575,7 @@ private[spark] class SparkArrayIndexOutOfBoundsException private(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass.orNull
override def getCondition: String = errorClass.orNull
override def getQueryContext: Array[QueryContext] = context
}

Expand All @@ -591,7 +591,7 @@ private[spark] class SparkSQLException(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass
override def getCondition: String = errorClass
}

/**
Expand All @@ -606,5 +606,5 @@ private[spark] class SparkSQLFeatureNotSupportedException(

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava

override def getErrorClass: String = errorClass
override def getCondition: String = errorClass
}
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ private[spark] object SparkThrowableHelper {
import ErrorMessageFormat._
format match {
case PRETTY => e.getMessage
case MINIMAL | STANDARD if e.getErrorClass == null =>
case MINIMAL | STANDARD if e.getCondition == null =>
toJsonString { generator =>
val g = generator.useDefaultPrettyPrinter()
g.writeStartObject()
Expand All @@ -92,7 +92,7 @@ private[spark] object SparkThrowableHelper {
g.writeEndObject()
}
case MINIMAL | STANDARD =>
val errorClass = e.getErrorClass
val errorClass = e.getCondition
toJsonString { generator =>
val g = generator.useDefaultPrettyPrinter()
g.writeStartObject()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ class StreamingQueryException private[sql](
s"""${classOf[StreamingQueryException].getName}: ${cause.getMessage}
|$queryDebugString""".stripMargin

override def getErrorClass: String = errorClass
override def getCondition: String = errorClass

override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava
}
Original file line number Diff line number Diff line change
Expand Up @@ -436,7 +436,7 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
val ex = intercept[SparkException] {
spark.read.format("avro").load(s"$dir.avro").collect()
}
assert(ex.getErrorClass.startsWith("FAILED_READ_FILE"))
assert(ex.getCondition.startsWith("FAILED_READ_FILE"))
checkError(
exception = ex.getCause.asInstanceOf[SparkArithmeticException],
condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION",
Expand Down
Loading

0 comments on commit ab3265e

Please sign in to comment.