This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 0110b1ed88d [SPARK-40810][SQL] Use SparkIllegalArgumentException instead of IllegalArgumentException in CreateDatabaseCommand & AlterDatabaseSetLocationCommand 0110b1ed88d is described below commit 0110b1ed88d3c140e1bdf04888740070f1d9d992 Author: panbingkun <pbk1...@gmail.com> AuthorDate: Wed Oct 19 10:33:22 2022 +0500 [SPARK-40810][SQL] Use SparkIllegalArgumentException instead of IllegalArgumentException in CreateDatabaseCommand & AlterDatabaseSetLocationCommand ### What changes were proposed in this pull request? This pr aims to use SparkIllegalArgumentException instead of IllegalArgumentException in CreateDatabaseCommand & AlterDatabaseSetLocationCommand. ### Why are the changes needed? When I work on https://issues.apache.org/jira/browse/SPARK-40790, I found when `location` is empty, DDL command(CreateDatabaseCommand & AlterDatabaseSetLocationCommand) throw IllegalArgumentException, it seem not to fit into the new error framework. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existed UT. Closes #38274 from panbingkun/setNamespaceLocation_error. Authored-by: panbingkun <pbk1...@gmail.com> Signed-off-by: Max Gekk <max.g...@gmail.com> --- core/src/main/resources/error/error-classes.json | 5 +++++ .../apache/spark/sql/errors/QueryExecutionErrors.scala | 6 ++++++ .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 10 +++++++++- .../execution/datasources/v2/DataSourceV2Strategy.scala | 9 +++++++++ .../command/AlterNamespaceSetLocationSuiteBase.scala | 12 ++++++++---- .../sql/execution/command/CreateNamespaceSuiteBase.scala | 15 ++++++++------- 6 files changed, 45 insertions(+), 12 deletions(-) diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json index 7f42d8acc53..0cfb6861c77 100644 --- a/core/src/main/resources/error/error-classes.json +++ b/core/src/main/resources/error/error-classes.json @@ -837,6 +837,11 @@ } } }, + "UNSUPPORTED_EMPTY_LOCATION" : { + "message" : [ + "Unsupported empty location." + ] + }, "UNSUPPORTED_FEATURE" : { "message" : [ "The feature is not supported:" 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 89c0cf5fafa..5edffc87b84 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 @@ -2657,4 +2657,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { "numElements" -> numElements.toString, "size" -> elementSize.toString)) } + + def unsupportedEmptyLocationError(): SparkIllegalArgumentException = { + new SparkIllegalArgumentException( + errorClass = "UNSUPPORTED_EMPTY_LOCATION", + messageParameters = Map.empty) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 56236f0d2ad..d00d07150b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.commons.lang3.StringUtils + import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils} @@ -27,7 +29,7 @@ import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, ResolveDe import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, LookupCatalog, SupportsNamespaces, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable => CreateTableV1, DataSource} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 @@ -131,6 +133,9 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) AlterDatabasePropertiesCommand(db, properties) case SetNamespaceLocation(DatabaseInSessionCatalog(db), location) if conf.useV1Command => + if (StringUtils.isEmpty(location)) { + throw QueryExecutionErrors.unsupportedEmptyLocationError() + } AlterDatabaseSetLocationCommand(db, location) case RenameTable(ResolvedV1TableOrViewIdentifier(oldIdent), newName, isView) => @@ -237,6 +242,9 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) val comment = c.properties.get(SupportsNamespaces.PROP_COMMENT) val location = c.properties.get(SupportsNamespaces.PROP_LOCATION) val newProperties = c.properties -- CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES + if (location.isDefined && location.get.isEmpty) { + throw QueryExecutionErrors.unsupportedEmptyLocationError() + } CreateDatabaseCommand(name, c.ifNotExists, location, comment, newProperties) case d @ DropNamespace(DatabaseInSessionCatalog(db), _, _) if conf.useV1Command => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 39ad51ffbe7..88dbe49c5af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import scala.collection.mutable +import org.apache.commons.lang3.StringUtils + import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, Strategy} import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable} @@ -350,6 +352,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat AlterNamespaceSetPropertiesExec(catalog.asNamespaceCatalog, ns, properties) :: Nil case SetNamespaceLocation(ResolvedNamespace(catalog, ns), location) => + if (StringUtils.isEmpty(location)) { + throw QueryExecutionErrors.unsupportedEmptyLocationError() + } AlterNamespaceSetPropertiesExec( catalog.asNamespaceCatalog, ns, @@ -362,6 +367,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat Map(SupportsNamespaces.PROP_COMMENT -> comment)) :: Nil case CreateNamespace(ResolvedNamespace(catalog, ns), ifNotExists, properties) => + val location = properties.get(SupportsNamespaces.PROP_LOCATION) + if (location.isDefined && location.get.isEmpty) { + throw QueryExecutionErrors.unsupportedEmptyLocationError() + } val finalProperties = properties.get(SupportsNamespaces.PROP_LOCATION).map { loc => properties + (SupportsNamespaces.PROP_LOCATION -> makeQualifiedDBObjectPath(loc)) }.getOrElse(properties) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala index ad4f27a8830..653a6d91891 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.command +import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.connector.catalog.SupportsNamespaces @@ -45,10 +46,13 @@ trait AlterNamespaceSetLocationSuiteBase extends QueryTest with DDLCommandTestUt val ns = s"$catalog.$namespace" withNamespace(ns) { sql(s"CREATE NAMESPACE $ns") - val message = intercept[IllegalArgumentException] { - sql(s"ALTER NAMESPACE $ns SET LOCATION ''") - }.getMessage - assert(message.contains("Can not create a Path from an empty string")) + val sqlText = s"ALTER NAMESPACE $ns SET LOCATION ''" + checkError( + exception = intercept[SparkIllegalArgumentException] { + sql(sqlText) + }, + errorClass = "UNSUPPORTED_EMPTY_LOCATION", + parameters = Map.empty) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala index ecd2f570aeb..1f0c3513dc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path +import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} @@ -68,15 +69,15 @@ trait CreateNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { // The generated temp path is not qualified. val path = tmpDir.getCanonicalPath assert(!path.startsWith("file:/")) - - val e = intercept[IllegalArgumentException] { - sql(s"CREATE NAMESPACE $ns LOCATION ''") - } - assert(e.getMessage.contains("Can not create a Path from an empty string")) - + val sqlText = s"CREATE NAMESPACE $ns LOCATION ''" + checkError( + exception = intercept[SparkIllegalArgumentException] { + sql(sqlText) + }, + errorClass = "UNSUPPORTED_EMPTY_LOCATION", + parameters = Map.empty) val uri = new Path(path).toUri sql(s"CREATE NAMESPACE $ns LOCATION '$uri'") - // Make sure the location is qualified. val expected = makeQualifiedPath(tmpDir.toString) assert("file" === expected.getScheme) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org