diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 4908a9b6c2e1..aa38f8b97470 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -246,6 +246,17 @@ "UNSUPPORTED_GROUPING_EXPRESSION" : { "message" : [ "grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup" ] }, + "UNSUPPORTED_SAVE_MODE" : { + "message" : [ "The save mode is not supported for: " ], + "subClass" : { + "EXISTENT_PATH" : { + "message" : [ "an existent path." ] + }, + "NON_EXISTENT_PATH" : { + "message" : [ "a not existent path." ] + } + } + }, "UNTYPED_SCALA_UDF" : { "message" : [ "You're using untyped Scala UDF, which does not have the input type information. Spark may blindly pass null to the Scala closure with primitive-type argument, and the closure will see the default value of the Java type for the null argument, e.g. `udf((x: Int) => x, IntegerType)`, the result is 0 for null input. To get rid of this error, you could:\n1. use typed Scala UDF APIs(without return type parameter), e.g. `udf((x: Int) => x)`\n2. use Java UDF APIs, e.g. `udf(new UDF1[String, Integer] { override def call(s: String): Integer = s.length() }, IntegerType)`, if input types are all non primitive\n3. set \"spark.sql.legacy.allowUntypedScalaUDF\" to true and use this API with caution" ] }, diff --git a/core/src/main/scala/org/apache/spark/ErrorInfo.scala b/core/src/main/scala/org/apache/spark/ErrorInfo.scala index a21f33e8833a..0447572bb1c2 100644 --- a/core/src/main/scala/org/apache/spark/ErrorInfo.scala +++ b/core/src/main/scala/org/apache/spark/ErrorInfo.scala @@ -80,9 +80,9 @@ private[spark] object SparkThrowableHelper { val errorSubInfo = subClass.getOrElse(subErrorClass, throw new IllegalArgumentException(s"Cannot find sub error class '$subErrorClass'")) val subMessageParameters = messageParameters.tail - "[" + errorClass + "." + subErrorClass + "] " + errorInfo.messageFormat + - String.format(errorSubInfo.messageFormat.replaceAll("<[a-zA-Z0-9_-]+>", "%s"), - subMessageParameters: _*) + "[" + errorClass + "." + subErrorClass + "] " + String.format((errorInfo.messageFormat + + errorSubInfo.messageFormat).replaceAll("<[a-zA-Z0-9_-]+>", "%s"), + subMessageParameters: _*) } else { "[" + errorClass + "] " + String.format( errorInfo.messageFormat.replaceAll("<[a-zA-Z0-9_-]+>", "%s"), 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 225315d3f02e..4b8d76e8e6fe 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 @@ -592,8 +592,13 @@ 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 74be483cd7c3..d773d4bd271b 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 338da79674e2..3857394b8bcb 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 @@ -21,8 +21,8 @@ import java.util.Locale import test.org.apache.spark.sql.connector.JavaSimpleWritableDataSource -import org.apache.spark.{SparkArithmeticException, SparkException, SparkIllegalStateException, SparkRuntimeException, SparkUnsupportedOperationException, SparkUpgradeException} -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest} +import org.apache.spark.{SparkArithmeticException, SparkException, SparkIllegalArgumentException, SparkIllegalStateException, SparkRuntimeException, SparkUnsupportedOperationException, SparkUpgradeException} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, SaveMode} import org.apache.spark.sql.catalyst.util.BadRecordException import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.QueryExecutionException @@ -33,6 +33,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.EXCEPTION import org.apache.spark.sql.types.{DecimalType, StructType, TimestampType} import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.util.Utils class QueryExecutionErrorsSuite extends QueryTest @@ -430,4 +431,30 @@ class QueryExecutionErrorsSuite matchMsg = true ) } + + 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) + } + checkErrorClass( + exception = e1, + errorClass = "UNSUPPORTED_SAVE_MODE", + errorSubClass = Some("NON_EXISTENT_PATH"), + msg = "The save mode NULL is not supported for: a not existent path.") + + Utils.createDirectory(path) + + val e2 = intercept[SparkIllegalArgumentException] { + val saveMode: SaveMode = null + Seq(1, 2).toDS().write.mode(saveMode).parquet(path.getAbsolutePath) + } + checkErrorClass( + exception = e2, + errorClass = "UNSUPPORTED_SAVE_MODE", + errorSubClass = Some("EXISTENT_PATH"), + msg = "The save mode NULL is not supported for: an existent path.") + } + } }