diff --git a/docs/sql-ref-syntax-ddl-alter-database.md b/docs/sql-ref-syntax-ddl-alter-database.md index 0ac003823643..727fedb19e23 100644 --- a/docs/sql-ref-syntax-ddl-alter-database.md +++ b/docs/sql-ref-syntax-ddl-alter-database.md @@ -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. @@ -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 @@ -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 diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index ff863565910d..54eff14b6d4d 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -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 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 bca2c8725394..dc43bd163659 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 @@ -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._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 59169dc51415..055fec02d2ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -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._ @@ -1098,4 +1098,22 @@ class SparkSqlAstBuilder extends AstBuilder { (ctx.LOCAL != null, finalStorage, Some(DDLUtils.HIVE_PROVIDER)) } + + /** + * 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) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala new file mode 100644 index 000000000000..243b51b09e3b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala @@ -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( + 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) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index eec79ad02e29..ea2736b2c126 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -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 @@ -51,6 +51,7 @@ trait RunnableCommand extends Command { } trait LeafRunnableCommand extends RunnableCommand with LeafLike[LogicalPlan] +trait UnaryRunnableCommand extends RunnableCommand with UnaryLike[LogicalPlan] /** * A physical operator that executes the run method of a `RunnableCommand` and diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala index c28c7b9db043..7f5b3de4865c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala @@ -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')" + 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) + ) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala new file mode 100644 index 000000000000..72d307c81666 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala @@ -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 '('") + ) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala new file mode 100644 index 000000000000..1d43cc593848 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -0,0 +1,121 @@ +/* + * 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.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces} +import org.apache.spark.sql.internal.SQLConf + +/** + * This base suite contains unified tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command + * that check V1 and V2 table catalogs. The tests that cannot run for all supported catalogs are + * located in more specific test suites: + * + * - V2 table catalog tests: + * `org.apache.spark.sql.execution.command.v2.AlterNamespaceUnsetPropertiesSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.AlterNamespaceUnsetPropertiesSuiteBase` + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterNamespaceUnsetPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterNamespaceUnsetPropertiesSuite` + */ +trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "ALTER NAMESPACE ... UNSET PROPERTIES" + + protected def namespace: String + + protected def getProperties(namespace: String): String = { + val propsRow = sql(s"DESCRIBE NAMESPACE EXTENDED $namespace") + .toDF("key", "value") + .where("key like 'Properties%'") + .collect() + assert(propsRow.length == 1) + propsRow(0).getString(1) + } + + test("namespace does not exist") { + val ns = "not_exist" + val e = intercept[AnalysisException] { + sql(s"ALTER NAMESPACE $catalog.$ns UNSET PROPERTIES ('d')") + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> s"`$ns`")) + } + + test("basic test") { + val ns = s"$catalog.$namespace" + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns") + assert(getProperties(ns) === "") + sql(s"ALTER NAMESPACE $ns SET PROPERTIES ('a'='a', 'b'='b', 'c'='c')") + assert(getProperties(ns) === "((a,a), (b,b), (c,c))") + sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") + assert(getProperties(ns) === "((a,a), (c,c))") + + // unset non-existent properties + // it will be successful, ignoring non-existent properties + sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") + assert(getProperties(ns) === "((a,a), (c,c))") + } + } + + test("test reserved properties") { + import SupportsNamespaces._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + val ns = s"$catalog.$namespace" + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns") + val sqlText = s"ALTER NAMESPACE $ns UNSET PROPERTIES ('$key')" + 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 = 37 + ns.length + key.length) + ) + } + } + } + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + withNamespace(ns) { + // Set the location explicitly because v2 catalog may not set the default location. + // Without this, `meta.get(key)` below may return null. + sql(s"CREATE NAMESPACE $ns LOCATION 'tmp/prop_test'") + assert(getProperties(ns) === "") + sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('$key')") + assert(getProperties(ns) === "", s"$key is a reserved namespace property and ignored") + val meta = spark.sessionState.catalogManager.catalog(catalog) + .asNamespaceCatalog.loadNamespaceMetadata(namespace.split('.')) + assert(!meta.get(key).contains("foo"), + "reserved properties should not have side effects") + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceUnsetPropertiesSuiteBase.scala new file mode 100644 index 000000000000..da7fdbba16b0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -0,0 +1,45 @@ +/* + * 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.v1 + +import org.apache.spark.sql.execution.command + +/** + * This base suite contains unified tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command + * that checks V1 table catalogs. The tests that cannot run for all V1 catalogs are located in more + * specific test suites: + * + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterNamespaceUnsetPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterNamespaceUnsetPropertiesSuite` + */ +trait AlterNamespaceUnsetPropertiesSuiteBase extends command.AlterNamespaceUnsetPropertiesSuiteBase + with command.TestsV1AndV2Commands { + override def namespace: String = "db" +} + +/** + * The class contains tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command to + * check V1 In-Memory table catalog. + */ +class AlterNamespaceUnsetPropertiesSuite extends AlterNamespaceUnsetPropertiesSuiteBase + with CommandSuiteBase { + override def commandVersion: String = + super[AlterNamespaceUnsetPropertiesSuiteBase].commandVersion +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesSuite.scala new file mode 100644 index 000000000000..352238eda2ea --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesSuite.scala @@ -0,0 +1,30 @@ +/* + * 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.v2 + +import org.apache.spark.sql.execution.command + +/** + * The class contains tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command to check V2 + * table catalogs. + */ +class AlterNamespaceUnsetPropertiesSuite extends command.AlterNamespaceUnsetPropertiesSuiteBase + with CommandSuiteBase { + + override def namespace: String = "ns1.ns2" +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceUnsetPropertiesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceUnsetPropertiesSuite.scala new file mode 100644 index 000000000000..22d833649fc6 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceUnsetPropertiesSuite.scala @@ -0,0 +1,29 @@ +/* + * 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.hive.execution.command + +import org.apache.spark.sql.execution.command.v1 + +/** + * The class contains tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command to check + * V1 Hive external table catalog. + */ +class AlterNamespaceUnsetPropertiesSuite extends v1.AlterNamespaceUnsetPropertiesSuiteBase + with CommandSuiteBase { + override def commandVersion: String = super[AlterNamespaceUnsetPropertiesSuiteBase].commandVersion +}