Skip to content
Closed
40 changes: 38 additions & 2 deletions docs/sql-ref-syntax-ddl-alter-database.md
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ license: |
`DATABASE`, `SCHEMA` and `NAMESPACE` are interchangeable and one can be used in place of the others. An error message
is issued if the database is not found in the system.

### ALTER PROPERTIES
### SET PROPERTIES
`ALTER DATABASE SET DBPROPERTIES` statement changes the properties associated with a database.
The specified property values override any existing value with the same property name.
This command is mostly used to record the metadata for a database and may be used for auditing purposes.
Expand All @@ -43,7 +43,25 @@ ALTER { DATABASE | SCHEMA | NAMESPACE } database_name

Specifies the name of the database to be altered.

### ALTER LOCATION
### UNSET PROPERTIES
`ALTER DATABASE UNSET DBPROPERTIES` statement unsets the properties associated with a database.
If the specified property key does not exist, the command will ignore it and finally succeed.
(available since Spark 4.0.0).

#### Syntax

```sql
ALTER { DATABASE | SCHEMA | NAMESPACE } database_name
UNSET { DBPROPERTIES | PROPERTIES } ( property_name [ , ... ] )
```

#### Parameters

* **database_name**

Specifies the name of the database to be altered.

### SET LOCATION
`ALTER DATABASE SET LOCATION` statement changes the default parent-directory where new tables will be added
for a database. Please note that it does not move the contents of the database's current directory to the newly
specified location or change the locations associated with any tables/partitions under the specified database
Expand Down Expand Up @@ -95,6 +113,24 @@ DESCRIBE DATABASE EXTENDED inventory;
| Location|file:/temp/spark-warehouse/new_inventory.db|
| Properties| ((Edit-date,01/01/2001), (Edited-by,John))|
+-------------------------+-------------------------------------------+

-- Alters the database to unset the property `Edited-by`
ALTER DATABASE inventory UNSET DBPROPERTIES ('Edited-by');

-- Verify that the property `Edited-by` has been unset.
DESCRIBE DATABASE EXTENDED inventory;
+-------------------------+-------------------------------------------+
|database_description_item| database_description_value|
+-------------------------+-------------------------------------------+
| Database Name| inventory|
| Description| |
| Location|file:/temp/spark-warehouse/new_inventory.db|
| Properties| ((Edit-date,01/01/2001)) |
+-------------------------+-------------------------------------------+

-- Alters the database to unset a non-existent property `non-existent`
-- Note: The command will ignore 'non-existent' and finally succeed
ALTER DATABASE inventory UNSET DBPROPERTIES ('non-existent');
```

### Related Statements
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,8 @@ statement
(WITH (DBPROPERTIES | PROPERTIES) propertyList))* #createNamespace
| ALTER namespace identifierReference
SET (DBPROPERTIES | PROPERTIES) propertyList #setNamespaceProperties
| ALTER namespace identifierReference
UNSET (DBPROPERTIES | PROPERTIES) propertyList #unsetNamespaceProperties
| ALTER namespace identifierReference
SET locationSpec #setNamespaceLocation
| DROP namespace (IF EXISTS)? identifierReference
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3671,7 +3671,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
}
}

private def cleanNamespaceProperties(
protected def cleanNamespaceProperties(
properties: Map[String, String],
ctx: ParserRuleContext): Map[String, String] = withOrigin(ctx) {
import SupportsNamespaces._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import org.antlr.v4.runtime.tree.TerminalNode

import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, SchemaEvolution, SchemaTypeEvolution, UnresolvedFunctionName, UnresolvedIdentifier}
import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, SchemaEvolution, SchemaTypeEvolution, UnresolvedFunctionName, UnresolvedIdentifier, UnresolvedNamespace}
import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.expressions.{Expression, Literal}
import org.apache.spark.sql.catalyst.parser._
Expand Down Expand Up @@ -1098,4 +1098,22 @@ class SparkSqlAstBuilder extends AstBuilder {

(ctx.LOCAL != null, finalStorage, Some(DDLUtils.HIVE_PROVIDER))
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have to put the parse process of UnsetNamespacePropertiesCommand here, because UnsetNamespacePropertiesCommand can only be defined within the module sql/core, and AstBuilder cannot access the class UnsetNamespacePropertiesCommand within the module

/**
* Create a [[UnsetNamespacePropertiesCommand]] command.
*
* Expected format:
* {{{
* ALTER (DATABASE|SCHEMA|NAMESPACE) database
* UNSET (DBPROPERTIES | PROPERTIES) ('key1', 'key2');
* }}}
*/
override def visitUnsetNamespaceProperties(
ctx: UnsetNamespacePropertiesContext): LogicalPlan = withOrigin(ctx) {
val properties = visitPropertyKeys(ctx.propertyList)
val cleanedProperties = cleanNamespaceProperties(properties.map(_ -> "").toMap, ctx).keys.toSeq
UnsetNamespacePropertiesCommand(
withIdentClause(ctx.identifierReference(), UnresolvedNamespace(_)),
cleanedProperties)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.command

import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper
import org.apache.spark.sql.connector.catalog.NamespaceChange

/**
* A command that unsets database/schema/namespace properties.
*
* The syntax of this command is:
* {{{
* ALTER (DATABASE|SCHEMA|NAMESPACE) ...
* UNSET (DBPROPERTIES|PROPERTIES) ('key1', 'key2', ...);
* }}}
*/
case class UnsetNamespacePropertiesCommand(
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have to define UnsetNamespacePropertiesCommand in module sql/core because it inherits UnaryRunnableCommand extends RunnableCommand and cannot be accessed in the module sql/catalyst (sql/core depends on sql/catalyst)

ident: LogicalPlan,
propKeys: Seq[String]) extends UnaryRunnableCommand {

override def run(sparkSession: SparkSession): Seq[Row] = {
val ResolvedNamespace(catalog, ns, _) = child
val changes = propKeys.map {
NamespaceChange.removeProperty
}
// If the property does not exist, the change should succeed.
catalog.asNamespaceCatalog.alterNamespace(ns.toArray, changes: _*)

Seq.empty
}

override def child: LogicalPlan = ident

override protected def withNewChildInternal(
newChild: LogicalPlan): UnsetNamespacePropertiesCommand =
copy(ident = newChild)
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, SupervisingCommand}
import org.apache.spark.sql.catalyst.trees.LeafLike
import org.apache.spark.sql.catalyst.trees.{LeafLike, UnaryLike}
import org.apache.spark.sql.connector.ExternalCommandRunner
import org.apache.spark.sql.execution.{CommandExecutionMode, ExplainMode, LeafExecNode, SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.metric.SQLMetric
Expand All @@ -51,6 +51,7 @@ trait RunnableCommand extends Command {
}

trait LeafRunnableCommand extends RunnableCommand with LeafLike[LogicalPlan]
trait UnaryRunnableCommand extends RunnableCommand with UnaryLike[LogicalPlan]
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because RunnableCommand is defined in the module sql/core, it seems that adding UnaryRunnableCommand here is more appropriate.


/**
* A physical operator that executes the run method of a `RunnableCommand` and
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,10 +83,19 @@ trait AlterNamespaceSetPropertiesSuiteBase extends QueryTest with DDLCommandTest
CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key =>
withNamespace(ns) {
sql(s"CREATE NAMESPACE $ns")
val exception = intercept[ParseException] {
sql(s"ALTER NAMESPACE $ns SET PROPERTIES ('$key'='dummyVal')")
}
assert(exception.getMessage.contains(s"$key is a reserved namespace property"))
val sqlText = s"ALTER NAMESPACE $ns SET PROPERTIES ('$key'='dummyVal')"
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not related to this PR,
Only let's use checkErrorMatchPVals to check for error-conditions

checkErrorMatchPVals(
exception = intercept[ParseException] {
sql(sqlText)
},
errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY",
parameters = Map("property" -> key, "msg" -> ".*"),
sqlState = None,
context = ExpectedContext(
fragment = sqlText,
start = 0,
stop = 46 + ns.length + key.length)
)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.execution.command

import org.apache.spark.SparkThrowable
import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace}
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.execution.SparkSqlParser
import org.apache.spark.sql.test.SharedSparkSession

class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedSparkSession {

private lazy val parser = new SparkSqlParser()

private def parseException(sqlText: String): SparkThrowable = {
intercept[ParseException](sql(sqlText).collect())
}

private def parsePlan(sqlText: String): LogicalPlan = {
parser.parsePlan(sqlText)
}

test("unset namespace properties") {
Seq("DATABASE", "SCHEMA", "NAMESPACE").foreach { nsToken =>
Seq("PROPERTIES", "DBPROPERTIES").foreach { propToken =>
comparePlans(
parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a', 'b', 'c')"),
UnsetNamespacePropertiesCommand(
UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c")))

comparePlans(
parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a')"),
UnsetNamespacePropertiesCommand(UnresolvedNamespace(Seq("a", "b", "c")), Seq("a")))
}
}
}

test("property values must not be set") {
val sql = "ALTER NAMESPACE my_db UNSET PROPERTIES('key_without_value', 'key_with_value'='x')"
checkError(
exception = parseException(sql),
errorClass = "_LEGACY_ERROR_TEMP_0035",
parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"),
context = ExpectedContext(
fragment = sql,
start = 0,
stop = 80))
}

test("not support clause - IF EXISTS") {
Seq("DATABASE", "SCHEMA", "NAMESPACE").foreach { nsToken =>
Seq("PROPERTIES", "DBPROPERTIES").foreach { propToken =>
val sql = s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')"
checkError(
exception = parseException(sql),
errorClass = "PARSE_SYNTAX_ERROR",
parameters = Map("error" -> "'IF'", "hint" -> ": missing '('")
)
}
}
}
}
Loading