Skip to content

Commit 56ac955

Browse files
committed
refine
1 parent 7188587 commit 56ac955

File tree

14 files changed

+80
-79
lines changed

14 files changed

+80
-79
lines changed

docs/sql-keywords.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -203,6 +203,7 @@ Below is a list of all the keywords in Spark SQL.
203203
<tr><td>OVERLAPS</td><td>reserved</td><td>non-reserved</td><td>reserved</td></tr>
204204
<tr><td>OVERLAY</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
205205
<tr><td>OVERWRITE</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
206+
<tr><td>OWNER</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
206207
<tr><td>PARTITION</td><td>non-reserved</td><td>non-reserved</td><td>reserved</td></tr>
207208
<tr><td>PARTITIONED</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
208209
<tr><td>PARTITIONS</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>

sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -98,7 +98,7 @@ statement
9898
| ALTER (database | NAMESPACE) multipartIdentifier
9999
SET locationSpec #setNamespaceLocation
100100
| ALTER (database | NAMESPACE) multipartIdentifier
101-
SET OWNER ownerType=(USER | ROLE | GROUP) IDENTIFIER #setNamespaceOwner
101+
SET OWNER ownerType=(USER | ROLE | GROUP) identifier #setNamespaceOwner
102102
| DROP (database | NAMESPACE) (IF EXISTS)? multipartIdentifier
103103
(RESTRICT | CASCADE)? #dropNamespace
104104
| SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)?

sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsNamespaces.java

Lines changed: 20 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
package org.apache.spark.sql.connector.catalog;
1919

2020
import org.apache.spark.annotation.Experimental;
21+
import org.apache.spark.sql.AnalysisException;
2122
import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException;
2223
import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
2324

@@ -64,9 +65,26 @@ public interface SupportsNamespaces extends CatalogPlugin {
6465
String PROP_OWNER_TYPE = "ownerType";
6566

6667
/**
67-
* The list of reserved namespace properties.
68+
* The list of namespace ownership properties, cannot be used in `CREATE` syntax.
69+
*
70+
* Only support in:
71+
*
72+
* {{
73+
* ALTER (DATABASE|SCHEMA|NAMESPACE) SET OWNER ...
74+
* }}
75+
*/
76+
List<String> OWNERSHIPS = Arrays.asList(PROP_OWNER_NAME, PROP_OWNER_TYPE);
77+
78+
/**
79+
* The list of immutable namespace properties, which can not be removed or changed directly by
80+
* the syntax:
81+
* {{
82+
* ALTER (DATABASE|SCHEMA|NAMESPACE) SET DBPROPERTIES(...)
83+
* }}
84+
*
85+
* They need specific syntax to modify
6886
*/
69-
List<String> RESERVED_PROPERTIES =
87+
List<String> REVERSED_PROPERTIES =
7088
Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_OWNER_NAME, PROP_OWNER_TYPE);
7189

7290
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala

Lines changed: 12 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -97,17 +97,20 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
9797
s"because view support in catalog has not been implemented yet")
9898

9999
case AlterNamespaceSetPropertiesStatement(NonSessionCatalog(catalog, nameParts), properties) =>
100-
val availableProps = properties -- RESERVED_PROPERTIES.asScala
101-
AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts, availableProps)
100+
if (properties.keySet.intersect(REVERSED_PROPERTIES.asScala.toSet).nonEmpty) {
101+
throw new AnalysisException(s"Cannot directly modify the reversed properties" +
102+
s" ${REVERSED_PROPERTIES.asScala.mkString("[", ",", "]")}.")
103+
}
104+
AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts, properties)
102105

103106
case AlterNamespaceSetLocationStatement(NonSessionCatalog(catalog, nameParts), location) =>
104107
AlterNamespaceSetProperties(catalog.asNamespaceCatalog, nameParts,
105108
Map(PROP_LOCATION -> location))
106109

107-
case AlterNamespaceSetOwnerStatement(NonSessionCatalog(catalog, nameParts), name, typ) =>
110+
case AlterNamespaceSetOwner(CatalogAndIdentifierParts(catalog, parts), name, typ) =>
108111
AlterNamespaceSetProperties(
109112
catalog.asNamespaceCatalog,
110-
nameParts,
113+
parts,
111114
Map(PROP_OWNER_NAME -> name, PROP_OWNER_TYPE -> typ))
112115

113116
case RenameTableStatement(NonSessionCatalog(catalog, oldName), newNameParts, isView) =>
@@ -185,12 +188,11 @@ class ResolveCatalogs(val catalogManager: CatalogManager)
185188
s"Can not specify catalog `${catalog.name}` for view ${viewName.quoted} " +
186189
s"because view support in catalog has not been implemented yet")
187190

188-
case c @ CreateNamespaceStatement(NonSessionCatalog(catalog, nameParts), _, _) =>
189-
CreateNamespace(
190-
catalog.asNamespaceCatalog,
191-
nameParts,
192-
c.ifNotExists,
193-
c.properties)
191+
case c @ CreateNamespaceStatement(NonSessionCatalog(catalog, nameParts), _, properties) =>
192+
if (properties.keySet.intersect(OWNERSHIPS.asScala.toSet).nonEmpty) {
193+
throw new AnalysisException("Cannot specify the ownership in CREATE NAMESPACE.")
194+
}
195+
CreateNamespace(catalog.asNamespaceCatalog, nameParts, c.ifNotExists, properties)
194196

195197
case DropNamespaceStatement(NonSessionCatalog(catalog, nameParts), ifExists, cascade) =>
196198
DropNamespace(catalog, nameParts, ifExists, cascade)

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -2580,7 +2580,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
25802580
}
25812581

25822582
/**
2583-
* Create an [[AlterNamespaceSetOwnerStatement]] logical plan.
2583+
* Create an [[AlterNamespaceSetOwner]] logical plan.
25842584
*
25852585
* For example:
25862586
* {{{
@@ -2589,9 +2589,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
25892589
*/
25902590
override def visitSetNamespaceOwner(ctx: SetNamespaceOwnerContext): LogicalPlan = {
25912591
withOrigin(ctx) {
2592-
AlterNamespaceSetOwnerStatement(
2592+
AlterNamespaceSetOwner(
25932593
visitMultipartIdentifier(ctx.multipartIdentifier),
2594-
ctx.IDENTIFIER.getText,
2594+
ctx.identifier.getText,
25952595
ctx.ownerType.getText)
25962596
}
25972597
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -385,7 +385,7 @@ case class AlterNamespaceSetLocationStatement(
385385
/**
386386
* ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET OWNER command, as parsed from SQL.
387387
*/
388-
case class AlterNamespaceSetOwnerStatement(
388+
case class AlterNamespaceSetOwner(
389389
namespace: Seq[String],
390390
ownerName: String,
391391
ownerType: String) extends ParsedStatement

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1218,14 +1218,14 @@ class DDLParserSuite extends AnalysisTest {
12181218
test("set namespace owner") {
12191219
comparePlans(
12201220
parsePlan("ALTER DATABASE a.b.c SET OWNER USER user1"),
1221-
AlterNamespaceSetOwnerStatement(Seq("a", "b", "c"), "user1", "USER"))
1221+
AlterNamespaceSetOwner(Seq("a", "b", "c"), "user1", "USER"))
12221222

12231223
comparePlans(
12241224
parsePlan("ALTER DATABASE a.b.c SET OWNER ROLE role1"),
1225-
AlterNamespaceSetOwnerStatement(Seq("a", "b", "c"), "role1", "ROLE"))
1225+
AlterNamespaceSetOwner(Seq("a", "b", "c"), "role1", "ROLE"))
12261226
comparePlans(
12271227
parsePlan("ALTER DATABASE a.b.c SET OWNER GROUP group1"),
1228-
AlterNamespaceSetOwnerStatement(Seq("a", "b", "c"), "group1", "GROUP"))
1228+
AlterNamespaceSetOwner(Seq("a", "b", "c"), "group1", "GROUP"))
12291229
}
12301230

12311231
test("show databases: basic") {

sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogT
2525
import org.apache.spark.sql.catalyst.plans.logical._
2626
import org.apache.spark.sql.catalyst.rules.Rule
2727
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, SupportsNamespaces, Table, TableCatalog, TableChange, V1Table}
28+
import org.apache.spark.sql.connector.catalog.SupportsNamespaces._
2829
import org.apache.spark.sql.connector.expressions.Transform
2930
import org.apache.spark.sql.execution.command._
3031
import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable}
@@ -172,6 +173,10 @@ class ResolveSessionCatalog(
172173
throw new AnalysisException(
173174
s"The database name is not valid: ${nameParts.quoted}")
174175
}
176+
if (properties.keySet.intersect(REVERSED_PROPERTIES.asScala.toSet).nonEmpty) {
177+
throw new AnalysisException(s"Cannot directly modify the reversed properties" +
178+
s" ${REVERSED_PROPERTIES.asScala.mkString("[", ",", "]")}.")
179+
}
175180
AlterDatabasePropertiesCommand(nameParts.head, properties)
176181

177182
case AlterNamespaceSetLocationStatement(SessionCatalog(_, nameParts), location) =>
@@ -181,13 +186,6 @@ class ResolveSessionCatalog(
181186
}
182187
AlterDatabaseSetLocationCommand(nameParts.head, location)
183188

184-
case AlterNamespaceSetOwnerStatement(SessionCatalog(_, nameParts), ownerName, ownerType) =>
185-
if (nameParts.length != 1) {
186-
throw new AnalysisException(
187-
s"The database name is not valid: ${nameParts.quoted}")
188-
}
189-
AlterDatabaseSetOwnerCommand(nameParts.head, ownerName, ownerType)
190-
191189
case RenameTableStatement(SessionCatalog(_, oldName), newNameParts, isView) =>
192190
AlterTableRenameCommand(oldName.asTableIdentifier, newNameParts.asTableIdentifier, isView)
193191

@@ -309,10 +307,12 @@ class ResolveSessionCatalog(
309307
throw new AnalysisException(
310308
s"The database name is not valid: ${nameParts.quoted}")
311309
}
312-
313-
val comment = c.properties.get(SupportsNamespaces.PROP_COMMENT)
314-
val location = c.properties.get(SupportsNamespaces.PROP_LOCATION)
315-
val newProperties = c.properties -- SupportsNamespaces.RESERVED_PROPERTIES.asScala
310+
if (c.properties.keySet.intersect(OWNERSHIPS.asScala.toSet).nonEmpty) {
311+
throw new AnalysisException("Cannot specify the ownership in CREATE DATABASE.")
312+
}
313+
val comment = c.properties.get(PROP_COMMENT)
314+
val location = c.properties.get(PROP_LOCATION)
315+
val newProperties = c.properties -- REVERSED_PROPERTIES.asScala
316316
CreateDatabaseCommand(nameParts.head, c.ifNotExists, location, comment, newProperties)
317317

318318
case d @ DropNamespaceStatement(SessionCatalog(_, nameParts), _, _) =>

sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala

Lines changed: 2 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -129,8 +129,7 @@ case class AlterDatabasePropertiesCommand(
129129
override def run(sparkSession: SparkSession): Seq[Row] = {
130130
val catalog = sparkSession.sessionState.catalog
131131
val db: CatalogDatabase = catalog.getDatabaseMetadata(databaseName)
132-
val availableProps = props -- RESERVED_PROPERTIES.asScala
133-
catalog.alterDatabase(db.copy(properties = db.properties ++ availableProps))
132+
catalog.alterDatabase(db.copy(properties = db.properties ++ props))
134133

135134
Seq.empty[Row]
136135
}
@@ -157,26 +156,6 @@ case class AlterDatabaseSetLocationCommand(databaseName: String, location: Strin
157156
}
158157
}
159158

160-
/**
161-
* A command for users to set ownership for a database
162-
* If the database does not exist, an error message will be issued to indicate the database
163-
* does not exist.
164-
* The syntax of using this command in SQL is:
165-
* {{{
166-
* ALTER (DATABASE|SCHEMA) database_name SET OWNER [USER|ROLE|GROUP] identityName
167-
* }}}
168-
*/
169-
case class AlterDatabaseSetOwnerCommand(databaseName: String, ownerName: String, ownerType: String)
170-
extends RunnableCommand {
171-
override def run(sparkSession: SparkSession): Seq[Row] = {
172-
val catalog = sparkSession.sessionState.catalog
173-
val database = catalog.getDatabaseMetadata(databaseName)
174-
val ownerships = Map(PROP_OWNER_NAME -> ownerName, PROP_OWNER_TYPE -> ownerType)
175-
catalog.alterDatabase(database.copy(properties = database.properties ++ ownerships))
176-
Seq.empty[Row]
177-
}
178-
}
179-
180159
/**
181160
* A command for users to show the name of the database, its comment (if one has been set), and its
182161
* root location on the filesystem. When extended is true, it also shows the database's properties
@@ -204,7 +183,7 @@ case class DescribeDatabaseCommand(
204183
Row("Owner Type", allDbProperties.getOrElse(PROP_OWNER_TYPE, "")) :: Nil
205184

206185
if (extended) {
207-
val properties = allDbProperties -- RESERVED_PROPERTIES.asScala
186+
val properties = allDbProperties -- REVERSED_PROPERTIES.asScala
208187
val propertiesStr =
209188
if (properties.isEmpty) {
210189
""

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,9 @@ import scala.collection.JavaConverters._
2222
import org.apache.spark.sql.{AnalysisException, Strategy}
2323
import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression}
2424
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
25-
import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetProperties, AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, RenameTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables}
25+
import org.apache.spark.sql.catalyst.plans.logical.{AlterNamespaceSetOwner, AlterNamespaceSetProperties, AlterTable, AppendData, CreateNamespace, CreateTableAsSelect, CreateV2Table, DeleteFromTable, DescribeNamespace, DescribeTable, DropNamespace, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, RefreshTable, RenameTable, Repartition, ReplaceTable, ReplaceTableAsSelect, SetCatalogAndNamespace, ShowCurrentNamespace, ShowNamespaces, ShowTableProperties, ShowTables}
2626
import org.apache.spark.sql.connector.catalog.{StagingTableCatalog, TableCapability}
27+
import org.apache.spark.sql.connector.catalog.SupportsNamespaces.{PROP_OWNER_NAME, PROP_OWNER_TYPE}
2728
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream}
2829
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
2930
import org.apache.spark.sql.execution.datasources.DataSourceStrategy

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -35,19 +35,21 @@ case class DescribeNamespaceExec(
3535
namespace: Seq[String],
3636
isExtended: Boolean) extends V2CommandExec {
3737
private val encoder = RowEncoder(StructType.fromAttributes(output)).resolveAndBind()
38+
import SupportsNamespaces._
3839

3940
override protected def run(): Seq[InternalRow] = {
4041
val rows = new ArrayBuffer[InternalRow]()
4142
val ns = namespace.toArray
4243
val metadata = catalog.loadNamespaceMetadata(ns)
4344

4445
rows += toCatalystRow("Namespace Name", ns.last)
45-
rows += toCatalystRow("Description", metadata.get(SupportsNamespaces.PROP_COMMENT))
46-
rows += toCatalystRow("Location", metadata.get(SupportsNamespaces.PROP_LOCATION))
46+
rows += toCatalystRow("Description", metadata.get(PROP_COMMENT))
47+
rows += toCatalystRow("Location", metadata.get(PROP_LOCATION))
48+
rows += toCatalystRow("Owner Name", metadata.get(PROP_OWNER_NAME))
49+
rows += toCatalystRow("Owner Type", metadata.get(PROP_OWNER_TYPE))
50+
4751
if (isExtended) {
48-
val properties =
49-
metadata.asScala.toSeq.filter(p =>
50-
!SupportsNamespaces.RESERVED_PROPERTIES.contains(p._1))
52+
val properties = metadata.asScala -- REVERSED_PROPERTIES.asScala
5153
if (properties.nonEmpty) {
5254
rows += toCatalystRow("Properties", properties.mkString("(", ",", ")"))
5355
}

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala

Lines changed: 4 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -225,15 +225,6 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf)
225225
override def alterNamespace(namespace: Array[String], changes: NamespaceChange*): Unit = {
226226
namespace match {
227227
case Array(db) =>
228-
// validate that this catalog's reserved properties are not removed
229-
changes.foreach {
230-
case remove: RemoveProperty
231-
if SupportsNamespaces.RESERVED_PROPERTIES.contains(remove.property) =>
232-
throw new UnsupportedOperationException(
233-
s"Cannot remove reserved property: ${remove.property}")
234-
case _ =>
235-
}
236-
237228
val metadata = catalog.getDatabaseMetadata(db).toMetadata
238229
catalog.alterDatabase(
239230
toCatalogDatabase(db, CatalogV2Util.applyNamespaceChanges(metadata, changes)))
@@ -263,6 +254,7 @@ class V2SessionCatalog(catalog: SessionCatalog, conf: SQLConf)
263254
}
264255

265256
private[sql] object V2SessionCatalog {
257+
import SupportsNamespaces._
266258

267259
/**
268260
* Convert v2 Transforms to v1 partition columns and an optional bucket spec.
@@ -292,12 +284,12 @@ private[sql] object V2SessionCatalog {
292284
defaultLocation: Option[URI] = None): CatalogDatabase = {
293285
CatalogDatabase(
294286
name = db,
295-
description = metadata.getOrDefault(SupportsNamespaces.PROP_COMMENT, ""),
296-
locationUri = Option(metadata.get(SupportsNamespaces.PROP_LOCATION))
287+
description = metadata.getOrDefault(PROP_COMMENT, ""),
288+
locationUri = Option(metadata.get(PROP_LOCATION))
297289
.map(CatalogUtils.stringToURI)
298290
.orElse(defaultLocation)
299291
.getOrElse(throw new IllegalArgumentException("Missing database location")),
300-
properties = metadata.asScala.toMap -- SupportsNamespaces.RESERVED_PROPERTIES.asScala)
292+
properties = metadata.asScala.toMap -- Seq(PROP_COMMENT, PROP_LOCATION))
301293
}
302294

303295
private implicit class CatalogDatabaseHelper(catalogDatabase: CatalogDatabase) {

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -757,7 +757,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite {
757757
actual: scala.collection.Map[String, String]): Unit = {
758758
// remove location and comment that are automatically added by HMS unless they are expected
759759
val toRemove =
760-
SupportsNamespaces.RESERVED_PROPERTIES.asScala.filter(expected.contains)
760+
SupportsNamespaces.REVERSED_PROPERTIES.asScala.filter(expected.contains)
761761
assert(expected -- toRemove === actual)
762762
}
763763

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala

Lines changed: 12 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -394,19 +394,25 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA
394394

395395
sql(s"CREATE DATABASE $db1")
396396
checkOwner(db1, currentUser, "USER")
397-
sql(s"ALTER DATABASE $db1 SET DBPROPERTIES ('a'='a', 'ownerName'='$owner'," +
398-
s" 'ownerType'='XXX')")
397+
sql(s"ALTER DATABASE $db1 SET DBPROPERTIES ('a'='a')")
399398
checkOwner(db1, currentUser, "USER")
399+
val e = intercept[AnalysisException](sql(s"ALTER DATABASE $db1 SET DBPROPERTIES ('a'='a',"
400+
+ s"'ownerName'='$owner','ownerType'='XXX')"))
401+
assert(e.getMessage.contains("ownerName"))
400402
sql(s"ALTER DATABASE $db1 SET OWNER ROLE $owner")
401403
checkOwner(db1, owner, "ROLE")
402404

403-
sql(s"CREATE DATABASE $db2 WITH DBPROPERTIES('ownerName'='$owner', 'ownerType'='XXX')")
404-
checkOwner(db2, currentUser, "USER")
405-
sql(s"ALTER DATABASE $db2 SET DBPROPERTIES ('a'='a', 'ownerName'='$owner'," +
406-
s" 'ownerType'='XXX')")
405+
val e2 = intercept[AnalysisException](
406+
sql(s"CREATE DATABASE $db2 WITH DBPROPERTIES('ownerName'='$owner', 'ownerType'='XXX')"))
407+
assert(e2.getMessage.contains("ownership"))
408+
sql(s"CREATE DATABASE $db2 WITH DBPROPERTIES('comment'='$owner')")
407409
checkOwner(db2, currentUser, "USER")
408410
sql(s"ALTER DATABASE $db2 SET OWNER GROUP $owner")
409411
checkOwner(db2, owner, "GROUP")
412+
sql(s"ALTER DATABASE $db2 SET OWNER GROUP `$owner`")
413+
checkOwner(db2, owner, "GROUP")
414+
sql(s"ALTER DATABASE $db2 SET OWNER GROUP OWNER")
415+
checkOwner(db2, "OWNER", "GROUP")
410416
} finally {
411417
catalog.reset()
412418
}

0 commit comments

Comments
 (0)