diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md
index b18855366bb2..4d8ae710fa73 100644
--- a/docs/sql-keywords.md
+++ b/docs/sql-keywords.md
@@ -197,6 +197,7 @@ Below is a list of all the keywords in Spark SQL.
| OVERLAPS | reserved | non-reserved | reserved |
| OVERLAY | non-reserved | non-reserved | non-reserved |
| OVERWRITE | non-reserved | non-reserved | non-reserved |
+ | OWNER | non-reserved | non-reserved | non-reserved |
| PARTITION | non-reserved | non-reserved | reserved |
| PARTITIONED | non-reserved | non-reserved | non-reserved |
| PARTITIONS | non-reserved | non-reserved | non-reserved |
diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index 751c7824694f..78b76efcddc7 100644
--- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -103,6 +103,8 @@ statement
SET (DBPROPERTIES | PROPERTIES) tablePropertyList #setNamespaceProperties
| ALTER namespace multipartIdentifier
SET locationSpec #setNamespaceLocation
+ | ALTER namespace multipartIdentifier
+ SET OWNER ownerType=(USER | ROLE | GROUP) identifier #setNamespaceOwner
| DROP namespace (IF EXISTS)? multipartIdentifier
(RESTRICT | CASCADE)? #dropNamespace
| SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)?
@@ -1095,6 +1097,7 @@ ansiNonReserved
| OVER
| OVERLAY
| OVERWRITE
+ | OWNER
| PARTITION
| PARTITIONED
| PARTITIONS
@@ -1347,6 +1350,7 @@ nonReserved
| OVERLAPS
| OVERLAY
| OVERWRITE
+ | OWNER
| PARTITION
| PARTITIONED
| PARTITIONS
@@ -1605,6 +1609,7 @@ OVER: 'OVER';
OVERLAPS: 'OVERLAPS';
OVERLAY: 'OVERLAY';
OVERWRITE: 'OVERWRITE';
+OWNER: 'OWNER';
PARTITION: 'PARTITION';
PARTITIONED: 'PARTITIONED';
PARTITIONS: 'PARTITIONS';
diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java
index 2e60487287f2..d58fe611b270 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java
@@ -64,9 +64,16 @@ public interface SupportsNamespaces extends CatalogPlugin {
String PROP_OWNER_TYPE = "ownerType";
/**
- * The list of reserved namespace properties.
+ * The list of reserved namespace properties, which can not be removed or changed directly by
+ * the syntax:
+ * {{
+ * ALTER NAMESPACE ... SET PROPERTIES ...
+ * }}
+ *
+ * They need specific syntax to modify
*/
- List RESERVED_PROPERTIES = Arrays.asList(PROP_COMMENT, PROP_LOCATION);
+ List RESERVED_PROPERTIES =
+ Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_OWNER_NAME, PROP_OWNER_TYPE);
/**
* Return a default namespace for the catalog.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 7f66dbf74cea..92e10dcf67ca 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -2532,6 +2532,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
case (PROP_COMMENT, _) =>
throw new ParseException(s"$PROP_COMMENT is a reserved namespace property, please use" +
s" the COMMENT clause to specify it.", ctx)
+ case (ownership, _) if ownership == PROP_OWNER_NAME || ownership == PROP_OWNER_TYPE =>
+ throw new ParseException(s"$ownership is a reserved namespace property , please use" +
+ " ALTER NAMESPACE ... SET OWNER ... to specify it.", ctx)
case _ =>
}
properties
@@ -3527,4 +3530,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier)
CommentOnTable(UnresolvedTable(nameParts), comment)
}
+
+ /**
+ * Create an [[AlterNamespaceSetOwner]] logical plan.
+ *
+ * For example:
+ * {{{
+ * ALTER (DATABASE|SCHEMA|NAMESPACE) namespace SET OWNER (USER|ROLE|GROUP) identityName;
+ * }}}
+ */
+ override def visitSetNamespaceOwner(ctx: SetNamespaceOwnerContext): LogicalPlan = {
+ withOrigin(ctx) {
+ val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier)
+ AlterNamespaceSetOwner(
+ UnresolvedNamespace(nameParts),
+ ctx.identifier.getText,
+ ctx.ownerType.getText)
+ }
+ }
+
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
index b414a969c92d..f1946dc7bb09 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
@@ -291,6 +291,16 @@ case class AlterNamespaceSetLocation(
override def children: Seq[LogicalPlan] = Seq(namespace)
}
+/**
+ * ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET OWNER command, as parsed from SQL.
+ */
+case class AlterNamespaceSetOwner(
+ child: LogicalPlan,
+ ownerName: String,
+ ownerType: String) extends Command {
+ override def children: Seq[LogicalPlan] = child :: Nil
+}
+
/**
* The logical plan of the SHOW NAMESPACES command that works for v2 catalogs.
*/
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index 2e815202c93c..32eaf610652e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -1305,6 +1305,19 @@ class DDLParserSuite extends AnalysisTest {
UnresolvedNamespace(Seq("a", "b", "c")), "/home/user/db"))
}
+ test("set namespace owner") {
+ comparePlans(
+ parsePlan("ALTER DATABASE a.b.c SET OWNER USER user1"),
+ AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "user1", "USER"))
+
+ comparePlans(
+ parsePlan("ALTER DATABASE a.b.c SET OWNER ROLE role1"),
+ AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "role1", "ROLE"))
+ comparePlans(
+ parsePlan("ALTER DATABASE a.b.c SET OWNER GROUP group1"),
+ AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "group1", "GROUP"))
+ }
+
test("show databases: basic") {
comparePlans(
parsePlan("SHOW DATABASES"),
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
index f8299eee8669..dcc5aec83e71 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
@@ -183,7 +183,7 @@ case class DescribeDatabaseCommand(
Row("Owner Type", allDbProperties.getOrElse(PROP_OWNER_TYPE, "")) :: Nil
if (extended) {
- val properties = allDbProperties -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE)
+ val properties = allDbProperties -- RESERVED_PROPERTIES.asScala
val propertiesStr =
if (properties.isEmpty) {
""
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala
index 0f69f85dd837..5af592cfd5f5 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateNamespaceExec.scala
@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.connector.catalog.SupportsNamespaces
+import org.apache.spark.util.Utils
/**
* Physical plan node for creating a namespace.
@@ -35,11 +36,14 @@ case class CreateNamespaceExec(
extends V2CommandExec {
override protected def run(): Seq[InternalRow] = {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+ import org.apache.spark.sql.connector.catalog.SupportsNamespaces._
val ns = namespace.toArray
if (!catalog.namespaceExists(ns)) {
try {
- catalog.createNamespace(ns, properties.asJava)
+ val ownership =
+ Map(PROP_OWNER_NAME -> Utils.getCurrentUserName(), PROP_OWNER_TYPE -> "USER")
+ catalog.createNamespace(ns, (properties ++ ownership).asJava)
} catch {
case _: NamespaceAlreadyExistsException if ifNotExists =>
logWarning(s"Namespace ${namespace.quoted} was created concurrently. Ignoring.")
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 8ee404e72b64..430f76e15b27 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
@@ -23,7 +23,7 @@ import org.apache.spark.sql.{AnalysisException, Strategy}
import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable}
import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression}
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
-import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetLocation, AlterNamespaceSetProperties, AlterTable, AppendData, CommentOnNamespace, CommentOnTable, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, RenameTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables}
+import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetLocation, AlterNamespaceSetOwner, AlterNamespaceSetProperties, AlterTable, AppendData, CommentOnNamespace, CommentOnTable, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, RenameTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables}
import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, TableChange}
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream}
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
@@ -249,6 +249,11 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
case r @ ShowTableProperties(DataSourceV2Relation(table, _, _), propertyKey) =>
ShowTablePropertiesExec(r.output, table, propertyKey) :: Nil
+ case AlterNamespaceSetOwner(ResolvedNamespace(catalog, namespace), name, typ) =>
+ val properties =
+ Map(SupportsNamespaces.PROP_OWNER_NAME -> name, SupportsNamespaces.PROP_OWNER_TYPE -> typ)
+ AlterNamespaceSetPropertiesExec(catalog, namespace, properties) :: Nil
+
case _ => Nil
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala
index 7ad872f10e84..979d740efa8f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala
@@ -35,6 +35,7 @@ case class DescribeNamespaceExec(
namespace: Seq[String],
isExtended: Boolean) extends V2CommandExec {
private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind()
+ import SupportsNamespaces._
override protected def run(): Seq[InternalRow] = {
val rows = new ArrayBuffer[InternalRow]()
@@ -42,14 +43,23 @@ case class DescribeNamespaceExec(
val metadata = catalog.loadNamespaceMetadata(ns)
rows += toCatalystRow("Namespace Name", ns.last)
- rows += toCatalystRow("Description", metadata.get(SupportsNamespaces.PROP_COMMENT))
- rows += toCatalystRow("Location", metadata.get(SupportsNamespaces.PROP_LOCATION))
+ Option(metadata.get(PROP_COMMENT)).foreach {
+ rows += toCatalystRow("Description", _)
+ }
+ Option(metadata.get(PROP_LOCATION)).foreach {
+ rows += toCatalystRow("Location", _)
+ }
+ Option(metadata.get(PROP_OWNER_NAME)).foreach {
+ rows += toCatalystRow("Owner Name", _)
+ }
+ Option(metadata.get(PROP_OWNER_TYPE)).foreach {
+ rows += toCatalystRow("Owner Type", _)
+ }
+
if (isExtended) {
- val properties =
- metadata.asScala.toSeq.filter(p =>
- !SupportsNamespaces.RESERVED_PROPERTIES.contains(p._1))
+ val properties = metadata.asScala -- RESERVED_PROPERTIES.asScala
if (properties.nonEmpty) {
- rows += toCatalystRow("Properties", properties.mkString("(", ",", ")"))
+ rows += toCatalystRow("Properties", properties.toSeq.mkString("(", ",", ")"))
}
}
rows
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
index 6e230109bf95..67d24536a5a4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
@@ -299,7 +299,8 @@ private[sql] object V2SessionCatalog {
.map(CatalogUtils.stringToURI)
.orElse(defaultLocation)
.getOrElse(throw new IllegalArgumentException("Missing database location")),
- properties = metadata.asScala.toMap -- SupportsNamespaces.RESERVED_PROPERTIES.asScala)
+ properties = metadata.asScala.toMap --
+ Seq(SupportsNamespaces.PROP_COMMENT, SupportsNamespaces.PROP_LOCATION))
}
private implicit class CatalogDatabaseHelper(catalogDatabase: CatalogDatabase) {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index ccf853fa1f10..593da7e81265 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION
import org.apache.spark.sql.sources.SimpleScanSource
import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.util.Utils
class DataSourceV2SQLSuite
extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = true)
@@ -884,7 +885,8 @@ class DataSourceV2SQLSuite
.isEmpty, s"$key is a reserved namespace property and ignored")
val meta =
catalog("testcat").asNamespaceCatalog.loadNamespaceMetadata(Array("reservedTest"))
- assert(meta.get(key) === null, "reserved properties should not have side effects")
+ assert(meta.get(key) == null || !meta.get(key).contains("foo"),
+ "reserved properties should not have side effects")
}
}
}
@@ -967,7 +969,9 @@ class DataSourceV2SQLSuite
assert(description === Seq(
Row("Namespace Name", "ns2"),
Row("Description", "test namespace"),
- Row("Location", "/tmp/ns_test")
+ Row("Location", "/tmp/ns_test"),
+ Row("Owner Name", Utils.getCurrentUserName()),
+ Row("Owner Type", "USER")
))
}
}
@@ -982,6 +986,8 @@ class DataSourceV2SQLSuite
Row("Namespace Name", "ns2"),
Row("Description", "test namespace"),
Row("Location", "/tmp/ns_test"),
+ Row("Owner Name", Utils.getCurrentUserName()),
+ Row("Owner Type", "USER"),
Row("Properties", "((a,b),(b,a),(c,c))")
))
}
@@ -1010,7 +1016,8 @@ class DataSourceV2SQLSuite
.isEmpty, s"$key is a reserved namespace property and ignored")
val meta =
catalog("testcat").asNamespaceCatalog.loadNamespaceMetadata(Array("reservedTest"))
- assert(meta.get(key) === null, "reserved properties should not have side effects")
+ assert(meta.get(key) == null || !meta.get(key).contains("foo"),
+ "reserved properties should not have side effects")
}
}
}
@@ -1025,7 +1032,25 @@ class DataSourceV2SQLSuite
assert(descriptionDf.collect() === Seq(
Row("Namespace Name", "ns2"),
Row("Description", "test namespace"),
- Row("Location", "/tmp/ns_test_2")
+ Row("Location", "/tmp/ns_test_2"),
+ Row("Owner Name", Utils.getCurrentUserName()),
+ Row("Owner Type", "USER")
+ ))
+ }
+ }
+
+ test("AlterNamespaceSetOwner using v2 catalog") {
+ withNamespace("testcat.ns1.ns2") {
+ sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " +
+ "'test namespace' LOCATION '/tmp/ns_test_3'")
+ sql("ALTER NAMESPACE testcat.ns1.ns2 SET OWNER ROLE adminRole")
+ val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2")
+ assert(descriptionDf.collect() === Seq(
+ Row("Namespace Name", "ns2"),
+ Row("Description", "test namespace"),
+ Row("Location", "/tmp/ns_test_3"),
+ Row("Owner Name", "adminRole"),
+ Row("Owner Type", "ROLE")
))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
index a02998856f78..8e599c952621 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
@@ -1010,31 +1010,18 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite {
assert(exc.getMessage.contains(testNs.quoted))
}
- test("alterNamespace: fail to remove location") {
+ test("alterNamespace: fail to remove reserved properties") {
val catalog = newCatalog()
catalog.createNamespace(testNs, emptyProps)
- val exc = intercept[UnsupportedOperationException] {
- catalog.alterNamespace(testNs, NamespaceChange.removeProperty("location"))
- }
-
- assert(exc.getMessage.contains("Cannot remove reserved property: location"))
-
- catalog.dropNamespace(testNs)
- }
+ SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { p =>
+ val exc = intercept[UnsupportedOperationException] {
+ catalog.alterNamespace(testNs, NamespaceChange.removeProperty(p))
+ }
+ assert(exc.getMessage.contains(s"Cannot remove reserved property: $p"))
- test("alterNamespace: fail to remove comment") {
- val catalog = newCatalog()
-
- catalog.createNamespace(testNs, Map("comment" -> "test db").asJava)
-
- val exc = intercept[UnsupportedOperationException] {
- catalog.alterNamespace(testNs, NamespaceChange.removeProperty("comment"))
}
-
- assert(exc.getMessage.contains("Cannot remove reserved property: comment"))
-
catalog.dropNamespace(testNs)
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index 4be09ada6972..b3f7fc4d0557 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -375,11 +375,14 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA
}
}
- private def checkOwner(db: String, expected: String): Unit = {
- val owner = sql(s"DESCRIBE DATABASE EXTENDED $db")
- .where("database_description_item='Owner Name'")
+ private def checkOwner(db: String, expectedOwnerName: String, expectedOwnerType: String): Unit = {
+ val df = sql(s"DESCRIBE DATABASE EXTENDED $db")
+ val owner = df.where("database_description_item='Owner Name'")
.collect().head.getString(1)
- assert(owner === expected)
+ val typ = df.where("database_description_item='Owner Type'")
+ .collect().head.getString(1)
+ assert(owner === expectedOwnerName)
+ assert(typ === expectedOwnerType)
}
test("Database Ownership") {
@@ -388,20 +391,29 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA
val db1 = "spark_29425_1"
val db2 = "spark_29425_2"
val owner = "spark_29425"
+ val currentUser = Utils.getCurrentUserName()
sql(s"CREATE DATABASE $db1")
- checkOwner(db1, Utils.getCurrentUserName())
+ checkOwner(db1, currentUser, "USER")
sql(s"ALTER DATABASE $db1 SET DBPROPERTIES ('a'='a')")
- checkOwner(db1, Utils.getCurrentUserName())
-
- // TODO: Specify ownership should be forbidden after we implement `SET OWNER` syntax
- sql(s"CREATE DATABASE $db2 WITH DBPROPERTIES('ownerName'='$owner')")
- checkOwner(db2, owner)
- sql(s"ALTER DATABASE $db2 SET DBPROPERTIES ('a'='a')")
- checkOwner(db2, owner)
- // TODO: Changing ownership should be forbidden after we implement `SET OWNER` syntax
- sql(s"ALTER DATABASE $db2 SET DBPROPERTIES ('ownerName'='a')")
- checkOwner(db2, "a")
+ checkOwner(db1, currentUser, "USER")
+ val e = intercept[ParseException](sql(s"ALTER DATABASE $db1 SET DBPROPERTIES ('a'='a',"
+ + s"'ownerName'='$owner','ownerType'='XXX')"))
+ assert(e.getMessage.contains("ownerName"))
+ sql(s"ALTER DATABASE $db1 SET OWNER ROLE $owner")
+ checkOwner(db1, owner, "ROLE")
+
+ val e2 = intercept[ParseException](
+ sql(s"CREATE DATABASE $db2 WITH DBPROPERTIES('ownerName'='$owner')"))
+ assert(e2.getMessage.contains("ownerName"))
+ sql(s"CREATE DATABASE $db2")
+ checkOwner(db2, currentUser, "USER")
+ sql(s"ALTER DATABASE $db2 SET OWNER GROUP $owner")
+ checkOwner(db2, owner, "GROUP")
+ sql(s"ALTER DATABASE $db2 SET OWNER GROUP `$owner`")
+ checkOwner(db2, owner, "GROUP")
+ sql(s"ALTER DATABASE $db2 SET OWNER GROUP OWNER")
+ checkOwner(db2, "OWNER", "GROUP")
} finally {
catalog.reset()
}