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. OVERLAPSreservednon-reservedreserved OVERLAYnon-reservednon-reservednon-reserved OVERWRITEnon-reservednon-reservednon-reserved + OWNERnon-reservednon-reservednon-reserved PARTITIONnon-reservednon-reservedreserved PARTITIONEDnon-reservednon-reservednon-reserved PARTITIONSnon-reservednon-reservednon-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() }