diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 2d55597b55fd4..90538c3e7767e 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -3961,7 +3961,8 @@ test_that("Call DataFrameWriter.save() API in Java without path and check argume # It makes sure that we can omit path argument in write.df API and then it calls # DataFrameWriter.save() without path. expect_error(write.df(df, source = "csv"), - "Error in save : illegal argument - Expected exactly one path to be specified") + paste("Error in save : org.apache.spark.SparkIllegalArgumentException:", + "Expected exactly one path to be specified")) expect_error(write.json(df, jsonPath), "Error in json : analysis error - Path file:.*already exists") expect_error(write.text(df, jsonPath), diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index d27a2cbde9728..edfc2421a73f1 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -3008,11 +3008,6 @@ ". If necessary set to false to bypass this error." ] }, - "_LEGACY_ERROR_TEMP_2001" : { - "message" : [ - " If necessary set to false to bypass this error" - ] - }, "_LEGACY_ERROR_TEMP_2002" : { "message" : [ "" @@ -3132,5 +3127,130 @@ "message" : [ " must override either or " ] + }, + "_LEGACY_ERROR_TEMP_2026" : { + "message" : [ + "Failed to convert value (class of ) with the type of to JSON." + ] + }, + "_LEGACY_ERROR_TEMP_2027" : { + "message" : [ + "Unexpected operator in correlated subquery" + ] + }, + "_LEGACY_ERROR_TEMP_2028" : { + "message" : [ + "This line should be unreachable" + ] + }, + "_LEGACY_ERROR_TEMP_2029" : { + "message" : [ + "Not supported rounding mode: " + ] + }, + "_LEGACY_ERROR_TEMP_2030" : { + "message" : [ + "Can not handle nested schema yet... plan " + ] + }, + "_LEGACY_ERROR_TEMP_2031" : { + "message" : [ + "The input external row cannot be null." + ] + }, + "_LEGACY_ERROR_TEMP_2032" : { + "message" : [ + "" + ] + }, + "_LEGACY_ERROR_TEMP_2033" : { + "message" : [ + "Unable to create database as failed to create its directory " + ] + }, + "_LEGACY_ERROR_TEMP_2034" : { + "message" : [ + "Unable to drop database as failed to delete its directory " + ] + }, + "_LEGACY_ERROR_TEMP_2035" : { + "message" : [ + "Unable to create table as failed to create its directory " + ] + }, + "_LEGACY_ERROR_TEMP_2036" : { + "message" : [ + "Unable to delete partition path " + ] + }, + "_LEGACY_ERROR_TEMP_2037" : { + "message" : [ + "Unable to drop table
as failed to delete its directory " + ] + }, + "_LEGACY_ERROR_TEMP_2038" : { + "message" : [ + "Unable to rename table to as failed to rename its directory " + ] + }, + "_LEGACY_ERROR_TEMP_2039" : { + "message" : [ + "Unable to create partition path " + ] + }, + "_LEGACY_ERROR_TEMP_2040" : { + "message" : [ + "Unable to rename partition path " + ] + }, + "_LEGACY_ERROR_TEMP_2041" : { + "message" : [ + " is not implemented" + ] + }, + "_LEGACY_ERROR_TEMP_2042" : { + "message" : [ + ". If necessary set to false to bypass this error." + ] + }, + "_LEGACY_ERROR_TEMP_2043" : { + "message" : [ + "- caused overflow" + ] + }, + "_LEGACY_ERROR_TEMP_2044" : { + "message" : [ + " caused overflow" + ] + }, + "_LEGACY_ERROR_TEMP_2045" : { + "message" : [ + "Unsupported table change: " + ] + }, + "_LEGACY_ERROR_TEMP_2046" : { + "message" : [ + "[BUG] Not a DataSourceRDDPartition: " + ] + }, + "_LEGACY_ERROR_TEMP_2047" : { + "message" : [ + "'path' is not specified" + ] + }, + "_LEGACY_ERROR_TEMP_2048" : { + "message" : [ + "Schema must be specified when creating a streaming source DataFrame. If some files already exist in the directory, then depending on the file format you may be able to create a static DataFrame on that directory with 'spark.read.load(directory)' and infer schema from it." + ] + }, + "_LEGACY_ERROR_TEMP_2049" : { + "message" : [ + "Data source does not support streamed " + ] + }, + "_LEGACY_ERROR_TEMP_2050" : { + "message" : [ + "Expected exactly one path to be specified, but got: " + ] } } 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 b15078d9b9153..341c3e72de8bf 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 @@ -296,7 +296,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { def ansiIllegalArgumentError(message: String): SparkIllegalArgumentException = { new SparkIllegalArgumentException( - errorClass = "_LEGACY_ERROR_TEMP_2001", + errorClass = "_LEGACY_ERROR_TEMP_2000", messageParameters = Map( "message" -> message, "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key))) @@ -495,88 +495,141 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { messageParameters = Map("className" -> className, "m1" -> m1, "m2" -> m2)) } - def failToConvertValueToJsonError(value: AnyRef, cls: Class[_], dataType: DataType): Throwable = { - new RuntimeException(s"Failed to convert value $value (class of $cls) " + - s"with the type of $dataType to JSON.") + def failToConvertValueToJsonError( + value: AnyRef, cls: Class[_], dataType: DataType): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2026", + messageParameters = Map( + "value" -> value.toString(), + "cls" -> cls.toString(), + "dataType" -> dataType.toString())) } - def unexpectedOperatorInCorrelatedSubquery(op: LogicalPlan, pos: String = ""): Throwable = { - new RuntimeException(s"Unexpected operator $op in correlated subquery" + pos) + def unexpectedOperatorInCorrelatedSubquery( + op: LogicalPlan, pos: String = ""): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2027", + messageParameters = Map("op" -> op.toString(), "pos" -> pos)) } - def unreachableError(err: String = ""): Throwable = { - new RuntimeException("This line should be unreachable" + err) + def unreachableError(err: String = ""): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2028", + messageParameters = Map("err" -> err)) } - def unsupportedRoundingMode(roundMode: BigDecimal.RoundingMode.Value): Throwable = { - new RuntimeException(s"Not supported rounding mode: $roundMode") + def unsupportedRoundingMode(roundMode: BigDecimal.RoundingMode.Value): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2029", + messageParameters = Map("roundMode" -> roundMode.toString())) } - def resolveCannotHandleNestedSchema(plan: LogicalPlan): Throwable = { - new RuntimeException(s"Can not handle nested schema yet... plan $plan") + def resolveCannotHandleNestedSchema(plan: LogicalPlan): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2030", + messageParameters = Map("plan" -> plan.toString())) } - def inputExternalRowCannotBeNullError(): RuntimeException = { - new RuntimeException("The input external row cannot be null.") + def inputExternalRowCannotBeNullError(): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2031", + messageParameters = Map.empty) } def fieldCannotBeNullMsg(index: Int, fieldName: String): String = { s"The ${index}th field '$fieldName' of input row cannot be null." } - def fieldCannotBeNullError(index: Int, fieldName: String): RuntimeException = { - new RuntimeException(fieldCannotBeNullMsg(index, fieldName)) + def fieldCannotBeNullError(index: Int, fieldName: String): SparkRuntimeException = { + new SparkRuntimeException( + errorClass = "_LEGACY_ERROR_TEMP_2032", + messageParameters = Map("fieldCannotBeNullMsg" -> fieldCannotBeNullMsg(index, fieldName))) } def unableToCreateDatabaseAsFailedToCreateDirectoryError( dbDefinition: CatalogDatabase, e: IOException): Throwable = { - new SparkException(s"Unable to create database ${dbDefinition.name} as failed " + - s"to create its directory ${dbDefinition.locationUri}", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2033", + messageParameters = Map( + "name" -> dbDefinition.name, + "locationUri" -> dbDefinition.locationUri.toString()), + cause = e) } def unableToDropDatabaseAsFailedToDeleteDirectoryError( dbDefinition: CatalogDatabase, e: IOException): Throwable = { - new SparkException(s"Unable to drop database ${dbDefinition.name} as failed " + - s"to delete its directory ${dbDefinition.locationUri}", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2034", + messageParameters = Map( + "name" -> dbDefinition.name, + "locationUri" -> dbDefinition.locationUri.toString()), + cause = e) } def unableToCreateTableAsFailedToCreateDirectoryError( table: String, defaultTableLocation: Path, e: IOException): Throwable = { - new SparkException(s"Unable to create table $table as failed " + - s"to create its directory $defaultTableLocation", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2035", + messageParameters = Map( + "table" -> table, + "defaultTableLocation" -> defaultTableLocation.toString()), + cause = e) } def unableToDeletePartitionPathError(partitionPath: Path, e: IOException): Throwable = { - new SparkException(s"Unable to delete partition path $partitionPath", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2036", + messageParameters = Map("partitionPath" -> partitionPath.toString()), + cause = e) } def unableToDropTableAsFailedToDeleteDirectoryError( table: String, dir: Path, e: IOException): Throwable = { - new SparkException(s"Unable to drop table $table as failed " + - s"to delete its directory $dir", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2037", + messageParameters = Map("table" -> table, "dir" -> dir.toString()), + cause = e) } def unableToRenameTableAsFailedToRenameDirectoryError( oldName: String, newName: String, oldDir: Path, e: IOException): Throwable = { - new SparkException(s"Unable to rename table $oldName to $newName as failed " + - s"to rename its directory $oldDir", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2038", + messageParameters = Map( + "oldName" -> oldName, + "newName" -> newName, + "oldDir" -> oldDir.toString()), + cause = e) } def unableToCreatePartitionPathError(partitionPath: Path, e: IOException): Throwable = { - new SparkException(s"Unable to create partition path $partitionPath", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2039", + messageParameters = Map("partitionPath" -> partitionPath.toString()), + cause = e) } def unableToRenamePartitionPathError(oldPartPath: Path, e: IOException): Throwable = { - new SparkException(s"Unable to rename partition path $oldPartPath", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2040", + messageParameters = Map("oldPartPath" -> oldPartPath.toString()), + cause = e) } - def methodNotImplementedError(methodName: String): Throwable = { - new UnsupportedOperationException(s"$methodName is not implemented") + def methodNotImplementedError(methodName: String): SparkUnsupportedOperationException = { + new SparkUnsupportedOperationException( + errorClass = "_LEGACY_ERROR_TEMP_2041", + messageParameters = Map("methodName" -> methodName)) } - def arithmeticOverflowError(e: ArithmeticException): ArithmeticException = { - new ArithmeticException(s"${e.getMessage}. If necessary set ${SQLConf.ANSI_ENABLED.key} " + - s"to false to bypass this error.") + def arithmeticOverflowError(e: ArithmeticException): SparkArithmeticException = { + new SparkArithmeticException( + errorClass = "_LEGACY_ERROR_TEMP_2042", + messageParameters = Map( + "message" -> e.getMessage, + "ansiConfig" -> toSQLConf(SQLConf.ANSI_ENABLED.key)), + context = Array.empty, + summary = "") } def arithmeticOverflowError( @@ -596,14 +649,24 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { summary = getSummary(context)) } - def unaryMinusCauseOverflowError(originValue: Int): ArithmeticException = { - arithmeticOverflowError(s"- ${toSQLValue(originValue, IntegerType)} caused overflow") + def unaryMinusCauseOverflowError(originValue: Int): SparkArithmeticException = { + new SparkArithmeticException( + errorClass = "_LEGACY_ERROR_TEMP_2043", + messageParameters = Map("sqlValue" -> toSQLValue(originValue, IntegerType)), + context = Array.empty, + summary = "") } def binaryArithmeticCauseOverflowError( - eval1: Short, symbol: String, eval2: Short): ArithmeticException = { - arithmeticOverflowError( - s"${toSQLValue(eval1, ShortType)} $symbol ${toSQLValue(eval2, ShortType)} caused overflow") + eval1: Short, symbol: String, eval2: Short): SparkArithmeticException = { + new SparkArithmeticException( + errorClass = "_LEGACY_ERROR_TEMP_2044", + messageParameters = Map( + "sqlValue1" -> toSQLValue(eval1, ShortType), + "symbol" -> symbol, + "sqlValue2" -> toSQLValue(eval2, ShortType)), + context = Array.empty, + summary = "") } def intervalArithmeticOverflowError( @@ -635,25 +698,29 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def unsupportedTableChangeError(e: IllegalArgumentException): Throwable = { - new SparkException(s"Unsupported table change: ${e.getMessage}", e) + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2045", + messageParameters = Map("message" -> e.getMessage), + cause = e) } def notADatasourceRDDPartitionError(split: Partition): Throwable = { - new SparkException(s"[BUG] Not a DataSourceRDDPartition: $split") + new SparkException( + errorClass = "_LEGACY_ERROR_TEMP_2046", + messageParameters = Map("split" -> split.toString()), + cause = null) } - def dataPathNotSpecifiedError(): Throwable = { - new IllegalArgumentException("'path' is not specified") + def dataPathNotSpecifiedError(): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2047", + messageParameters = Map.empty) } - def createStreamingSourceNotSpecifySchemaError(): Throwable = { - new IllegalArgumentException( - s""" - |Schema must be specified when creating a streaming source DataFrame. If some - |files already exist in the directory, then depending on the file format you - |may be able to create a static DataFrame on that directory with - |'spark.read.load(directory)' and infer schema from it. - """.stripMargin) + def createStreamingSourceNotSpecifySchemaError(): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2048", + messageParameters = Map.empty) } def inferDateWithLegacyTimeParserError(): Throwable = { @@ -663,14 +730,16 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { } def streamedOperatorUnsupportedByDataSourceError( - className: String, operator: String): Throwable = { - new UnsupportedOperationException( - s"Data source $className does not support streamed $operator") + className: String, operator: String): SparkUnsupportedOperationException = { + new SparkUnsupportedOperationException( + errorClass = "_LEGACY_ERROR_TEMP_2049", + messageParameters = Map("className" -> className, "operator" -> operator)) } - def multiplePathsSpecifiedError(allPaths: Seq[String]): Throwable = { - new IllegalArgumentException("Expected exactly one path to be specified, but " + - s"got: ${allPaths.mkString(", ")}") + def multiplePathsSpecifiedError(allPaths: Seq[String]): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "_LEGACY_ERROR_TEMP_2050", + messageParameters = Map("paths" -> allPaths.mkString(", "))) } def failedToFindDataSourceError(provider: String, error: Throwable): Throwable = {