diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 2e32482328a7e..2d16ec50a6612 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -184,6 +184,17 @@ "UNSUPPORTED_OPERATION" : { "message" : [ "The operation is not supported: " ] }, + "UNSUPPORTED_SAVE_MODE" : { + "message" : [ "The save mode 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" diff --git a/core/src/main/scala/org/apache/spark/ErrorInfo.scala b/core/src/main/scala/org/apache/spark/ErrorInfo.scala index 99e081ebb45ea..fa2e1034692ac 100644 --- a/core/src/main/scala/org/apache/spark/ErrorInfo.scala +++ b/core/src/main/scala/org/apache/spark/ErrorInfo.scala @@ -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") @@ -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 = { 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 9e29acf04d2ef..6c6139d2ccc92 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 @@ -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 = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 74be483cd7c37..d773d4bd271b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -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) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index e6ce1d7008039..73c5b12849a1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -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 { @@ -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.") + } + } }