-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-37444][SQL] ALTER NAMESPACE ... SET LOCATION should handle empty location consistently across v1 and v2 command #34686
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -21,8 +21,9 @@ import scala.collection.JavaConverters._ | |||
| import scala.collection.mutable | ||||
|
|
||||
| import org.apache.spark.sql.{SparkSession, Strategy} | ||||
| import org.apache.spark.sql.catalyst.{expressions, SQLConfHelper} | ||||
| import org.apache.spark.sql.catalyst.analysis.{ResolvedDBObjectName, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable} | ||||
| import org.apache.spark.sql.catalyst.expressions | ||||
| import org.apache.spark.sql.catalyst.catalog.CatalogUtils | ||||
| import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning, EmptyRow, Expression, Literal, NamedExpression, PredicateHelper, SubqueryExpression} | ||||
| import org.apache.spark.sql.catalyst.planning.PhysicalOperation | ||||
| import org.apache.spark.sql.catalyst.plans.logical._ | ||||
|
|
@@ -44,7 +45,8 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap | |||
| import org.apache.spark.storage.StorageLevel | ||||
| import org.apache.spark.unsafe.types.UTF8String | ||||
|
|
||||
| class DataSourceV2Strategy(session: SparkSession) extends Strategy with PredicateHelper { | ||||
| class DataSourceV2Strategy(session: SparkSession) extends Strategy | ||||
| with PredicateHelper with SQLConfHelper { | ||||
|
||||
|
|
||||
| import DataSourceV2Implicits._ | ||||
| import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ | ||||
|
|
@@ -311,10 +313,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat | |||
| AlterNamespaceSetPropertiesExec(catalog.asNamespaceCatalog, ns, properties) :: Nil | ||||
|
|
||||
| case SetNamespaceLocation(ResolvedNamespace(catalog, ns), location) => | ||||
| val nsPath = CatalogUtils.makeQualifiedNamespacePath( | ||||
| CatalogUtils.stringToURI(location), conf.warehousePath, session.sharedState.hadoopConf) | ||||
|
||||
| SharedState.setWarehousePathConf(confClone, hadoopConfClone, qualified) |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1294,13 +1294,25 @@ class DataSourceV2SQLSuite | |
| assert(descriptionDf.collect() === Seq( | ||
| Row("Namespace Name", "ns2"), | ||
| Row(SupportsNamespaces.PROP_COMMENT.capitalize, "test namespace"), | ||
| Row(SupportsNamespaces.PROP_LOCATION.capitalize, "/tmp/ns_test_2"), | ||
| Row(SupportsNamespaces.PROP_LOCATION.capitalize, "file:/tmp/ns_test_2"), | ||
| Row(SupportsNamespaces.PROP_OWNER.capitalize, defaultUser), | ||
| Row("Properties", "")) | ||
| ) | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-37444: ALTER NAMESPACE .. SET LOCATION using v2 catalog with empty location") { | ||
| val ns = "testcat.ns1.ns2" | ||
| withNamespace(ns) { | ||
| sql(s"CREATE NAMESPACE IF NOT EXISTS $ns COMMENT " + | ||
| "'test namespace' LOCATION '/tmp/ns_test_1'") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. shall we do the same thing for CREATE NAMESPACE? We can do it in a separated PR.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, we should fix this. I will do it in a separate PR. Thanks! |
||
| val e = intercept[IllegalArgumentException] { | ||
| sql(s"ALTER DATABASE $ns SET LOCATION ''") | ||
| } | ||
| assert(e.getMessage.contains("Can not create a Path from an empty string")) | ||
| } | ||
| } | ||
|
|
||
| private def testShowNamespaces( | ||
| sqlText: String, | ||
| expected: Seq[String]): Unit = { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
These are copied from
SessionCatalog.scala