Skip to content

Commit bcf07cb

Browse files
yaooqinncloud-fan
authored andcommitted
[SPARK-30018][SQL] Support ALTER DATABASE SET OWNER syntax
### What changes were proposed in this pull request? In this pull request, we are going to support `SET OWNER` syntax for databases and namespaces, ```sql ALTER (DATABASE|SCHEME|NAMESPACE) database_name SET OWNER [USER|ROLE|GROUP] user_or_role_group; ``` Before this commit 332e252, we didn't care much about ownerships for the catalog objects. In 332e252, we determined to use properties to store ownership staff, and temporarily used `alter database ... set dbproperties ...` to support switch ownership of a database. This PR aims to use the formal syntax to replace it. In hive, `ownerName/Type` are fields of the database objects, also they can be normal properties. ``` create schema test1 with dbproperties('ownerName'='yaooqinn') ``` The create/alter database syntax will not change the owner to `yaooqinn` but store it in parameters. e.g. ``` +----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+ | db_name | comment | location | owner_name | owner_type | parameters | +----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+ | test1 | | hdfs://quickstart.cloudera:8020/user/hive/warehouse/test1.db | anonymous | USER | {ownerName=yaooqinn} | +----------+----------+---------------------------------------------------------------+-------------+-------------+-----------------------+--+ ``` In this pull request, because we let the `ownerName` become reversed, so it will neither change the owner nor store in dbproperties, just be omitted silently. ## Why are the changes needed? Formal syntax support for changing database ownership ### Does this PR introduce any user-facing change? yes, add a new syntax ### How was this patch tested? add unit tests Closes #26775 from yaooqinn/SPARK-30018. Authored-by: Kent Yao <yaooqinn@hotmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent afd70a0 commit bcf07cb

File tree

14 files changed

+152
-50
lines changed

14 files changed

+152
-50
lines changed

docs/sql-keywords.md

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -197,6 +197,7 @@ Below is a list of all the keywords in Spark SQL.
197197
<tr><td>OVERLAPS</td><td>reserved</td><td>non-reserved</td><td>reserved</td></tr>
198198
<tr><td>OVERLAY</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
199199
<tr><td>OVERWRITE</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
200+
<tr><td>OWNER</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
200201
<tr><td>PARTITION</td><td>non-reserved</td><td>non-reserved</td><td>reserved</td></tr>
201202
<tr><td>PARTITIONED</td><td>non-reserved</td><td>non-reserved</td><td>non-reserved</td></tr>
202203
<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: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -103,6 +103,8 @@ statement
103103
SET (DBPROPERTIES | PROPERTIES) tablePropertyList #setNamespaceProperties
104104
| ALTER namespace multipartIdentifier
105105
SET locationSpec #setNamespaceLocation
106+
| ALTER namespace multipartIdentifier
107+
SET OWNER ownerType=(USER | ROLE | GROUP) identifier #setNamespaceOwner
106108
| DROP namespace (IF EXISTS)? multipartIdentifier
107109
(RESTRICT | CASCADE)? #dropNamespace
108110
| SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)?
@@ -1098,6 +1100,7 @@ ansiNonReserved
10981100
| OVER
10991101
| OVERLAY
11001102
| OVERWRITE
1103+
| OWNER
11011104
| PARTITION
11021105
| PARTITIONED
11031106
| PARTITIONS
@@ -1350,6 +1353,7 @@ nonReserved
13501353
| OVERLAPS
13511354
| OVERLAY
13521355
| OVERWRITE
1356+
| OWNER
13531357
| PARTITION
13541358
| PARTITIONED
13551359
| PARTITIONS
@@ -1608,6 +1612,7 @@ OVER: 'OVER';
16081612
OVERLAPS: 'OVERLAPS';
16091613
OVERLAY: 'OVERLAY';
16101614
OVERWRITE: 'OVERWRITE';
1615+
OWNER: 'OWNER';
16111616
PARTITION: 'PARTITION';
16121617
PARTITIONED: 'PARTITIONED';
16131618
PARTITIONS: 'PARTITIONS';

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

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -64,9 +64,16 @@ public interface SupportsNamespaces extends CatalogPlugin {
6464
String PROP_OWNER_TYPE = "ownerType";
6565

6666
/**
67-
* The list of reserved namespace properties.
67+
* The list of reserved namespace properties, which can not be removed or changed directly by
68+
* the syntax:
69+
* {{
70+
* ALTER NAMESPACE ... SET PROPERTIES ...
71+
* }}
72+
*
73+
* They need specific syntax to modify
6874
*/
69-
List<String> RESERVED_PROPERTIES = Arrays.asList(PROP_COMMENT, PROP_LOCATION);
75+
List<String> RESERVED_PROPERTIES =
76+
Arrays.asList(PROP_COMMENT, PROP_LOCATION, PROP_OWNER_NAME, PROP_OWNER_TYPE);
7077

7178
/**
7279
* Return a default namespace for the catalog.

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

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2535,6 +2535,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
25352535
case (PROP_COMMENT, _) =>
25362536
throw new ParseException(s"$PROP_COMMENT is a reserved namespace property, please use" +
25372537
s" the COMMENT clause to specify it.", ctx)
2538+
case (ownership, _) if ownership == PROP_OWNER_NAME || ownership == PROP_OWNER_TYPE =>
2539+
throw new ParseException(s"$ownership is a reserved namespace property , please use" +
2540+
" ALTER NAMESPACE ... SET OWNER ... to specify it.", ctx)
25382541
case _ =>
25392542
}
25402543
properties
@@ -3562,4 +3565,23 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
35623565
val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier)
35633566
CommentOnTable(UnresolvedTable(nameParts), comment)
35643567
}
3568+
3569+
/**
3570+
* Create an [[AlterNamespaceSetOwner]] logical plan.
3571+
*
3572+
* For example:
3573+
* {{{
3574+
* ALTER (DATABASE|SCHEMA|NAMESPACE) namespace SET OWNER (USER|ROLE|GROUP) identityName;
3575+
* }}}
3576+
*/
3577+
override def visitSetNamespaceOwner(ctx: SetNamespaceOwnerContext): LogicalPlan = {
3578+
withOrigin(ctx) {
3579+
val nameParts = visitMultipartIdentifier(ctx.multipartIdentifier)
3580+
AlterNamespaceSetOwner(
3581+
UnresolvedNamespace(nameParts),
3582+
ctx.identifier.getText,
3583+
ctx.ownerType.getText)
3584+
}
3585+
}
3586+
35653587
}

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

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -291,6 +291,16 @@ case class AlterNamespaceSetLocation(
291291
override def children: Seq[LogicalPlan] = Seq(namespace)
292292
}
293293

294+
/**
295+
* ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET OWNER command, as parsed from SQL.
296+
*/
297+
case class AlterNamespaceSetOwner(
298+
child: LogicalPlan,
299+
ownerName: String,
300+
ownerType: String) extends Command {
301+
override def children: Seq[LogicalPlan] = child :: Nil
302+
}
303+
294304
/**
295305
* The logical plan of the SHOW NAMESPACES command that works for v2 catalogs.
296306
*/

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

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1343,6 +1343,19 @@ class DDLParserSuite extends AnalysisTest {
13431343
UnresolvedNamespace(Seq("a", "b", "c")), "/home/user/db"))
13441344
}
13451345

1346+
test("set namespace owner") {
1347+
comparePlans(
1348+
parsePlan("ALTER DATABASE a.b.c SET OWNER USER user1"),
1349+
AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "user1", "USER"))
1350+
1351+
comparePlans(
1352+
parsePlan("ALTER DATABASE a.b.c SET OWNER ROLE role1"),
1353+
AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "role1", "ROLE"))
1354+
comparePlans(
1355+
parsePlan("ALTER DATABASE a.b.c SET OWNER GROUP group1"),
1356+
AlterNamespaceSetOwner(UnresolvedNamespace(Seq("a", "b", "c")), "group1", "GROUP"))
1357+
}
1358+
13461359
test("show databases: basic") {
13471360
comparePlans(
13481361
parsePlan("SHOW DATABASES"),

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -183,7 +183,7 @@ case class DescribeDatabaseCommand(
183183
Row("Owner Type", allDbProperties.getOrElse(PROP_OWNER_TYPE, "")) :: Nil
184184

185185
if (extended) {
186-
val properties = allDbProperties -- Seq(PROP_OWNER_NAME, PROP_OWNER_TYPE)
186+
val properties = allDbProperties -- RESERVED_PROPERTIES.asScala
187187
val propertiesStr =
188188
if (properties.isEmpty) {
189189
""

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

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow
2323
import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException
2424
import org.apache.spark.sql.catalyst.expressions.Attribute
2525
import org.apache.spark.sql.connector.catalog.SupportsNamespaces
26+
import org.apache.spark.util.Utils
2627

2728
/**
2829
* Physical plan node for creating a namespace.
@@ -35,11 +36,14 @@ case class CreateNamespaceExec(
3536
extends V2CommandExec {
3637
override protected def run(): Seq[InternalRow] = {
3738
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
39+
import org.apache.spark.sql.connector.catalog.SupportsNamespaces._
3840

3941
val ns = namespace.toArray
4042
if (!catalog.namespaceExists(ns)) {
4143
try {
42-
catalog.createNamespace(ns, properties.asJava)
44+
val ownership =
45+
Map(PROP_OWNER_NAME -> Utils.getCurrentUserName(), PROP_OWNER_TYPE -> "USER")
46+
catalog.createNamespace(ns, (properties ++ ownership).asJava)
4347
} catch {
4448
case _: NamespaceAlreadyExistsException if ifNotExists =>
4549
logWarning(s"Namespace ${namespace.quoted} was created concurrently. Ignoring.")

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

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import org.apache.spark.sql.{AnalysisException, Strategy}
2323
import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable}
2424
import org.apache.spark.sql.catalyst.expressions.{And, PredicateHelper, SubqueryExpression}
2525
import org.apache.spark.sql.catalyst.planning.PhysicalOperation
26-
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}
26+
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}
2727
import org.apache.spark.sql.connector.catalog.{Identifier, StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, TableChange}
2828
import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream}
2929
import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan}
@@ -249,6 +249,11 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper {
249249
case r @ ShowTableProperties(DataSourceV2Relation(table, _, _), propertyKey) =>
250250
ShowTablePropertiesExec(r.output, table, propertyKey) :: Nil
251251

252+
case AlterNamespaceSetOwner(ResolvedNamespace(catalog, namespace), name, typ) =>
253+
val properties =
254+
Map(SupportsNamespaces.PROP_OWNER_NAME -> name, SupportsNamespaces.PROP_OWNER_TYPE -> typ)
255+
AlterNamespaceSetPropertiesExec(catalog, namespace, properties) :: Nil
256+
252257
case _ => Nil
253258
}
254259
}

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

Lines changed: 16 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -35,21 +35,31 @@ 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+
Option(metadata.get(PROP_COMMENT)).foreach {
47+
rows += toCatalystRow("Description", _)
48+
}
49+
Option(metadata.get(PROP_LOCATION)).foreach {
50+
rows += toCatalystRow("Location", _)
51+
}
52+
Option(metadata.get(PROP_OWNER_NAME)).foreach {
53+
rows += toCatalystRow("Owner Name", _)
54+
}
55+
Option(metadata.get(PROP_OWNER_TYPE)).foreach {
56+
rows += toCatalystRow("Owner Type", _)
57+
}
58+
4759
if (isExtended) {
48-
val properties =
49-
metadata.asScala.toSeq.filter(p =>
50-
!SupportsNamespaces.RESERVED_PROPERTIES.contains(p._1))
60+
val properties = metadata.asScala -- RESERVED_PROPERTIES.asScala
5161
if (properties.nonEmpty) {
52-
rows += toCatalystRow("Properties", properties.mkString("(", ",", ")"))
62+
rows += toCatalystRow("Properties", properties.toSeq.mkString("(", ",", ")"))
5363
}
5464
}
5565
rows

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -299,7 +299,8 @@ private[sql] object V2SessionCatalog {
299299
.map(CatalogUtils.stringToURI)
300300
.orElse(defaultLocation)
301301
.getOrElse(throw new IllegalArgumentException("Missing database location")),
302-
properties = metadata.asScala.toMap -- SupportsNamespaces.RESERVED_PROPERTIES.asScala)
302+
properties = metadata.asScala.toMap --
303+
Seq(SupportsNamespaces.PROP_COMMENT, SupportsNamespaces.PROP_LOCATION))
303304
}
304305

305306
private implicit class CatalogDatabaseHelper(catalogDatabase: CatalogDatabase) {

sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala

Lines changed: 29 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION
3030
import org.apache.spark.sql.sources.SimpleScanSource
3131
import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType}
3232
import org.apache.spark.sql.util.CaseInsensitiveStringMap
33+
import org.apache.spark.util.Utils
3334

3435
class DataSourceV2SQLSuite
3536
extends InsertIntoTests(supportsDynamicOverwrite = true, includeSQLOnlyTests = true)
@@ -886,7 +887,8 @@ class DataSourceV2SQLSuite
886887
.isEmpty, s"$key is a reserved namespace property and ignored")
887888
val meta =
888889
catalog("testcat").asNamespaceCatalog.loadNamespaceMetadata(Array("reservedTest"))
889-
assert(meta.get(key) === null, "reserved properties should not have side effects")
890+
assert(meta.get(key) == null || !meta.get(key).contains("foo"),
891+
"reserved properties should not have side effects")
890892
}
891893
}
892894
}
@@ -969,7 +971,9 @@ class DataSourceV2SQLSuite
969971
assert(description === Seq(
970972
Row("Namespace Name", "ns2"),
971973
Row("Description", "test namespace"),
972-
Row("Location", "/tmp/ns_test")
974+
Row("Location", "/tmp/ns_test"),
975+
Row("Owner Name", Utils.getCurrentUserName()),
976+
Row("Owner Type", "USER")
973977
))
974978
}
975979
}
@@ -984,6 +988,8 @@ class DataSourceV2SQLSuite
984988
Row("Namespace Name", "ns2"),
985989
Row("Description", "test namespace"),
986990
Row("Location", "/tmp/ns_test"),
991+
Row("Owner Name", Utils.getCurrentUserName()),
992+
Row("Owner Type", "USER"),
987993
Row("Properties", "((a,b),(b,a),(c,c))")
988994
))
989995
}
@@ -1012,7 +1018,8 @@ class DataSourceV2SQLSuite
10121018
.isEmpty, s"$key is a reserved namespace property and ignored")
10131019
val meta =
10141020
catalog("testcat").asNamespaceCatalog.loadNamespaceMetadata(Array("reservedTest"))
1015-
assert(meta.get(key) === null, "reserved properties should not have side effects")
1021+
assert(meta.get(key) == null || !meta.get(key).contains("foo"),
1022+
"reserved properties should not have side effects")
10161023
}
10171024
}
10181025
}
@@ -1027,7 +1034,25 @@ class DataSourceV2SQLSuite
10271034
assert(descriptionDf.collect() === Seq(
10281035
Row("Namespace Name", "ns2"),
10291036
Row("Description", "test namespace"),
1030-
Row("Location", "/tmp/ns_test_2")
1037+
Row("Location", "/tmp/ns_test_2"),
1038+
Row("Owner Name", Utils.getCurrentUserName()),
1039+
Row("Owner Type", "USER")
1040+
))
1041+
}
1042+
}
1043+
1044+
test("AlterNamespaceSetOwner using v2 catalog") {
1045+
withNamespace("testcat.ns1.ns2") {
1046+
sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " +
1047+
"'test namespace' LOCATION '/tmp/ns_test_3'")
1048+
sql("ALTER NAMESPACE testcat.ns1.ns2 SET OWNER ROLE adminRole")
1049+
val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2")
1050+
assert(descriptionDf.collect() === Seq(
1051+
Row("Namespace Name", "ns2"),
1052+
Row("Description", "test namespace"),
1053+
Row("Location", "/tmp/ns_test_3"),
1054+
Row("Owner Name", "adminRole"),
1055+
Row("Owner Type", "ROLE")
10311056
))
10321057
}
10331058
}

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

Lines changed: 6 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -995,31 +995,18 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite {
995995
assert(exc.getMessage.contains(testNs.quoted))
996996
}
997997

998-
test("alterNamespace: fail to remove location") {
998+
test("alterNamespace: fail to remove reserved properties") {
999999
val catalog = newCatalog()
10001000

10011001
catalog.createNamespace(testNs, emptyProps)
10021002

1003-
val exc = intercept[UnsupportedOperationException] {
1004-
catalog.alterNamespace(testNs, NamespaceChange.removeProperty("location"))
1005-
}
1006-
1007-
assert(exc.getMessage.contains("Cannot remove reserved property: location"))
1008-
1009-
catalog.dropNamespace(testNs)
1010-
}
1003+
SupportsNamespaces.RESERVED_PROPERTIES.asScala.foreach { p =>
1004+
val exc = intercept[UnsupportedOperationException] {
1005+
catalog.alterNamespace(testNs, NamespaceChange.removeProperty(p))
1006+
}
1007+
assert(exc.getMessage.contains(s"Cannot remove reserved property: $p"))
10111008

1012-
test("alterNamespace: fail to remove comment") {
1013-
val catalog = newCatalog()
1014-
1015-
catalog.createNamespace(testNs, Map("comment" -> "test db").asJava)
1016-
1017-
val exc = intercept[UnsupportedOperationException] {
1018-
catalog.alterNamespace(testNs, NamespaceChange.removeProperty("comment"))
10191009
}
1020-
1021-
assert(exc.getMessage.contains("Cannot remove reserved property: comment"))
1022-
10231010
catalog.dropNamespace(testNs)
10241011
}
10251012
}

0 commit comments

Comments
 (0)