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
5 changes: 5 additions & 0 deletions core/src/main/resources/error/error-classes.json
Original file line number Diff line number Diff line change
Expand Up @@ -709,6 +709,11 @@
}
}
},
"UNSUPPORTED_EMPTY_LOCATION" : {
"message" : [
"Unsupported empty location."
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

empty location is a bit misleading. This is not a location with no files/directories, it's empty string. How about

EMPTY_STRING_AS_LOCATION: Can not reference a location with an empty string.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@panbingkun Could you open a follow up PR and change this, and address below comment, please.

]
},
"UNSUPPORTED_FEATURE" : {
"message" : [
"The feature is not supported:"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand All @@ -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
Expand Down Expand Up @@ -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) =>
Expand Down Expand Up @@ -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 =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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,
Expand All @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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 ''"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can move this test to the base suite now, as the behavior is consistent between v1 and v2 tables.

checkError(
exception = intercept[SparkIllegalArgumentException] {
sql(sqlText)
},
errorClass = "UNSUPPORTED_EMPTY_LOCATION",
parameters = Map.empty)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.ParseException
Expand Down Expand Up @@ -67,15 +68,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 ''"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto

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)
Expand Down