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
11 changes: 11 additions & 0 deletions core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,17 @@
"UNSUPPORTED_OPERATION" : {
"message" : [ "The operation is not supported: <operation>" ]
},
"UNSUPPORTED_SAVE_MODE" : {
"message" : [ "The save mode <saveMode> is not supported for: " ],
"subClass" : {
"EXISTENT_PATH" : {
"message" : [ "an existent path." ]
},
"NON_EXISTENT_PATH" : {
"message" : [ "a non-existent path." ]
}
}
},
"WRITING_JOB_ABORTED" : {
"message" : [ "Writing job aborted" ],
"sqlState" : "40000"
Expand Down
34 changes: 31 additions & 3 deletions core/src/main/scala/org/apache/spark/ErrorInfo.scala
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,30 @@ import com.fasterxml.jackson.module.scala.DefaultScalaModule

import org.apache.spark.util.Utils

/**
* Information associated with an error subclass.
*
* @param message C-style message format compatible with printf.
* The error message is constructed by concatenating the lines with newlines.
*/
private[spark] case class ErrorSubInfo(message: Seq[String]) {
// For compatibility with multi-line error messages
@JsonIgnore
val messageFormat: String = message.mkString("\n")
}

/**
* Information associated with an error class.
*
* @param sqlState SQLSTATE associated with this class.
* @param subClass SubClass associated with this class.
* @param message C-style message format compatible with printf.
* The error message is constructed by concatenating the lines with newlines.
*/
private[spark] case class ErrorInfo(message: Seq[String], sqlState: Option[String]) {
private[spark] case class ErrorInfo(
message: Seq[String],
subClass: Option[Map[String, ErrorSubInfo]],
sqlState: Option[String]) {
// For compatibility with multi-line error messages
@JsonIgnore
val messageFormat: String = message.mkString("\n")
Expand All @@ -61,13 +77,25 @@ private[spark] object SparkThrowableHelper {
queryContext: String = ""): String = {
val errorInfo = errorClassToInfoMap.getOrElse(errorClass,
throw new IllegalArgumentException(s"Cannot find error class '$errorClass'"))
val (displayMessageParameters, displayFormat) = if (errorInfo.subClass.isDefined) {
val subClass = errorInfo.subClass.get
val subErrorClass = messageParameters.head
val errorSubInfo = subClass.getOrElse(subErrorClass,
throw new IllegalArgumentException(s"Cannot find sub error class '$subErrorClass'"))
val subMessageParameters = messageParameters.tail
(subMessageParameters, errorInfo.messageFormat + errorSubInfo.messageFormat)
} else {
(messageParameters, errorInfo.messageFormat)
}
val displayMessage = String.format(
displayFormat.replaceAll("<[a-zA-Z0-9_-]+>", "%s"),
displayMessageParameters : _*)
val displayQueryContext = if (queryContext.isEmpty) {
""
} else {
s"\n$queryContext"
}
String.format(errorInfo.messageFormat.replaceAll("<[a-zA-Z0-9_-]+>", "%s"),
messageParameters: _*) + displayQueryContext
s"$displayMessage$displayQueryContext"
}

def getSqlState(errorClass: String): String = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -652,8 +652,13 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
""".stripMargin)
}

def unsupportedSaveModeError(saveMode: String, pathExists: Boolean): Throwable = {
new IllegalStateException(s"unsupported save mode $saveMode ($pathExists)")
def saveModeUnsupportedError(saveMode: Any, pathExists: Boolean): Throwable = {
pathExists match {
case true => new SparkIllegalArgumentException(errorClass = "UNSUPPORTED_SAVE_MODE",
messageParameters = Array("EXISTENT_PATH", toSQLValue(saveMode, StringType)))
case _ => new SparkIllegalArgumentException(errorClass = "UNSUPPORTED_SAVE_MODE",
messageParameters = Array("NON_EXISTENT_PATH", toSQLValue(saveMode, StringType)))
}
}

def cannotClearOutputDirectoryError(staticPrefixPath: Path): Throwable = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ case class InsertIntoHadoopFsRelationCommand(
case (SaveMode.Ignore, exists) =>
!exists
case (s, exists) =>
throw QueryExecutionErrors.unsupportedSaveModeError(s.toString, exists)
throw QueryExecutionErrors.saveModeUnsupportedError(s, exists)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,15 @@

package org.apache.spark.sql.errors

import org.apache.spark.{SparkArithmeticException, SparkException, SparkRuntimeException, SparkUnsupportedOperationException, SparkUpgradeException}
import org.apache.spark.sql.{DataFrame, QueryTest}
import org.apache.spark.{SparkArithmeticException, SparkException, SparkIllegalArgumentException, SparkRuntimeException, SparkUnsupportedOperationException, SparkUpgradeException}
import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode}
import org.apache.spark.sql.execution.datasources.orc.OrcTest
import org.apache.spark.sql.execution.datasources.parquet.ParquetTest
import org.apache.spark.sql.functions.{lit, lower, struct, sum}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.EXCEPTION
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.Utils

class QueryExecutionErrorsSuite extends QueryTest
with ParquetTest with OrcTest with SharedSparkSession {
Expand Down Expand Up @@ -265,4 +266,24 @@ class QueryExecutionErrorsSuite extends QueryTest
assert(e.getMessage ===
"Datetime operation overflow: add 1000000 YEAR to TIMESTAMP '2022-03-09 01:02:03'.")
}

test("UNSUPPORTED_SAVE_MODE: unsupported null saveMode whether the path exists or not") {
withTempPath { path =>
val e1 = intercept[SparkIllegalArgumentException] {
val saveMode: SaveMode = null
Seq(1, 2).toDS().write.mode(saveMode).parquet(path.getAbsolutePath)
}
assert(e1.getErrorClass === "UNSUPPORTED_SAVE_MODE")
assert(e1.getMessage === "The save mode NULL is not supported for: a non-existent path.")

Utils.createDirectory(path)

val e2 = intercept[SparkIllegalArgumentException] {
val saveMode: SaveMode = null
Seq(1, 2).toDS().write.mode(saveMode).parquet(path.getAbsolutePath)
}
assert(e2.getErrorClass === "UNSUPPORTED_SAVE_MODE")
assert(e2.getMessage === "The save mode NULL is not supported for: an existent path.")
}
}
}