diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 7d29a9ed0ae7..c10e0bbfde21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -30,8 +30,10 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.connector.catalog.SupportsNamespaces.PROP_OWNER import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils /** * An in-memory (ephemeral) implementation of the system catalog. @@ -124,7 +126,9 @@ class InMemoryCatalog( throw QueryExecutionErrors.unableToCreateDatabaseAsFailedToCreateDirectoryError( dbDefinition, e) } - catalog.put(dbDefinition.name, new DatabaseDesc(dbDefinition)) + val newDb = dbDefinition.copy( + properties = dbDefinition.properties ++ Map(PROP_OWNER -> Utils.getCurrentUserName)) + catalog.put(dbDefinition.name, new DatabaseDesc(newDb)) } } 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 b92be01f9a3c..9354614c4121 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 @@ -1770,38 +1770,6 @@ class DDLParserSuite extends AnalysisTest { "location" -> "/home/user/db"))) } - test("set namespace properties") { - comparePlans( - parsePlan("ALTER DATABASE a.b.c SET PROPERTIES ('a'='a', 'b'='b', 'c'='c')"), - SetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a", "b" -> "b", "c" -> "c"))) - - comparePlans( - parsePlan("ALTER SCHEMA a.b.c SET PROPERTIES ('a'='a')"), - SetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a"))) - - comparePlans( - parsePlan("ALTER NAMESPACE a.b.c SET PROPERTIES ('b'='b')"), - SetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Map("b" -> "b"))) - - comparePlans( - parsePlan("ALTER DATABASE a.b.c SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')"), - SetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a", "b" -> "b", "c" -> "c"))) - - comparePlans( - parsePlan("ALTER SCHEMA a.b.c SET DBPROPERTIES ('a'='a')"), - SetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a"))) - - comparePlans( - parsePlan("ALTER NAMESPACE a.b.c SET DBPROPERTIES ('b'='b')"), - SetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Map("b" -> "b"))) - } - test("analyze table statistics") { comparePlans(parsePlan("analyze table a.b.c compute statistics"), AnalyzeTable( 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 4c5a001ec076..8b3ae25e3009 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 @@ -1224,54 +1224,6 @@ class DataSourceV2SQLSuite } } - test("ALTER NAMESPACE .. SET PROPERTIES using v2 catalog") { - withNamespace("testcat.ns1.ns2") { - sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + - "'test namespace' LOCATION '/tmp/ns_test' WITH PROPERTIES ('a'='a','b'='b','c'='c')") - sql("ALTER NAMESPACE testcat.ns1.ns2 SET PROPERTIES ('a'='b','b'='a')") - val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") - assert(descriptionDf.collect() === Seq( - Row("Namespace Name", "ns2"), - Row(SupportsNamespaces.PROP_COMMENT.capitalize, "test namespace"), - Row(SupportsNamespaces.PROP_LOCATION.capitalize, "file:/tmp/ns_test"), - Row(SupportsNamespaces.PROP_OWNER.capitalize, defaultUser), - Row("Properties", "((a,b), (b,a), (c,c))")) - ) - } - } - - test("ALTER NAMESPACE .. SET PROPERTIES reserved properties") { - import SupportsNamespaces._ - withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => - withNamespace("testcat.reservedTest") { - sql("CREATE NAMESPACE testcat.reservedTest") - val exception = intercept[ParseException] { - sql(s"ALTER NAMESPACE testcat.reservedTest SET PROPERTIES ('$key'='dummyVal')") - } - assert(exception.getMessage.contains(s"$key is a reserved namespace property")) - } - } - } - withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { - CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => - withNamespace("testcat.reservedTest") { - sql(s"CREATE NAMESPACE testcat.reservedTest") - sql(s"ALTER NAMESPACE testcat.reservedTest SET PROPERTIES ('$key'='foo')") - assert(sql("DESC NAMESPACE EXTENDED testcat.reservedTest") - .toDF("k", "v") - .where("k='Properties'") - .where("v=''") - .count == 1, s"$key is a reserved namespace property and ignored") - val meta = - catalog("testcat").asNamespaceCatalog.loadNamespaceMetadata(Array("reservedTest")) - assert(meta.get(key) == null || !meta.get(key).contains("foo"), - "reserved properties should not have side effects") - } - } - } - } - private def testShowNamespaces( sqlText: String, expected: Seq[String]): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesParserSuite.scala new file mode 100644 index 000000000000..868dc275b8a4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesParserSuite.scala @@ -0,0 +1,49 @@ +/* + * 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.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.SetNamespaceProperties + +class AlterNamespaceSetPropertiesParserSuite extends AnalysisTest { + test("set namespace properties") { + Seq("DATABASE", "SCHEMA", "NAMESPACE").foreach { nsToken => + Seq("PROPERTIES", "DBPROPERTIES").foreach { propToken => + comparePlans( + parsePlan(s"ALTER $nsToken a.b.c SET $propToken ('a'='a', 'b'='b', 'c'='c')"), + SetNamespaceProperties( + UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a", "b" -> "b", "c" -> "c"))) + + comparePlans( + parsePlan(s"ALTER $nsToken a.b.c SET $propToken ('a'='a')"), + SetNamespaceProperties( + UnresolvedNamespace(Seq("a", "b", "c")), Map("a" -> "a"))) + } + } + } + + test("property values must be set") { + val e = intercept[ParseException] { + parsePlan("ALTER NAMESPACE my_db SET PROPERTIES('key_without_value', 'key_with_value'='x')") + } + assert(e.getMessage.contains( + "Operation not allowed: Values must be specified for key(s): [key_without_value]")) + } +} 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 new file mode 100644 index 000000000000..c33795c836a1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala @@ -0,0 +1,117 @@ +/* + * 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 ... SET 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.AlterNamespaceSetPropertiesSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.AlterNamespaceSetPropertiesSuiteBase` + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterNamespaceSetPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterNamespaceSetPropertiesSuite` + */ +trait AlterNamespaceSetPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "ALTER NAMESPACE ... SET PROPERTIES" + + protected def namespace: String + + protected def notFoundMsgPrefix: String + + test("Namespace does not exist") { + val ns = "not_exist" + val message = intercept[AnalysisException] { + sql(s"ALTER DATABASE $catalog.$ns SET PROPERTIES ('d'='d')") + }.getMessage + assert(message.contains(s"$notFoundMsgPrefix '$ns' not found")) + } + + 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 DATABASE $ns SET PROPERTIES ('d'='d')") + assert(getProperties(ns) === "((a,a), (b,b), (c,c), (d,d))") + } + } + + test("test with properties set while creating namespace") { + val ns = s"$catalog.$namespace" + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns WITH PROPERTIES ('a'='a','b'='b','c'='c')") + assert(getProperties(ns) === "((a,a), (b,b), (c,c))") + sql(s"ALTER NAMESPACE $ns SET PROPERTIES ('a'='b', 'b'='a')") + assert(getProperties(ns) === "((a,b), (b,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 exception = intercept[ParseException] { + sql(s"ALTER NAMESPACE $ns SET PROPERTIES ('$key'='dummyVal')") + } + assert(exception.getMessage.contains(s"$key is a reserved namespace property")) + } + } + } + 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'") + assert(getProperties(ns) === "") + sql(s"ALTER NAMESPACE $ns SET PROPERTIES ('$key'='foo')") + 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") + } + } + } + } + + protected def getProperties(namespace: String): String = { + val propsRow = sql(s"DESCRIBE NAMESPACE EXTENDED $namespace") + .toDF("key", "value") + .where(s"key like 'Properties%'") + .collect() + assert(propsRow.length == 1) + propsRow(0).getString(1) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index b935c8b93a5b..44d68584a0cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -59,12 +59,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { ShowCurrentNamespaceCommand()) } - test("alter database - property values must be set") { - assertUnsupported( - sql = "ALTER DATABASE my_db SET DBPROPERTIES('key_without_value', 'key_with_value'='x')", - containsThesePhrases = Seq("key_without_value")) - } - test("insert overwrite directory") { val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a" parser.parsePlan(v1) match { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 1cb7868a7819..56de04591992 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -751,7 +751,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("Alter/Describe Database") { + test("Describe Database") { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -760,9 +760,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { val dbNameWithoutBackTicks = cleanIdentifier(dbName) val location = getDBPath(dbNameWithoutBackTicks) - sql(s"CREATE DATABASE $dbName") - - sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + sql(s"CREATE DATABASE $dbName WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c')") checkAnswer( sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") @@ -771,36 +769,12 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { Row("Comment", "") :: Row("Location", CatalogUtils.URIToString(location)) :: Row("Properties", "((a,a), (b,b), (c,c))") :: Nil) - - sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") - - checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") - .where("key not like 'Owner%'"), // filter for consistency with in-memory catalog - Row("Namespace Name", dbNameWithoutBackTicks) :: - Row("Comment", "") :: - Row("Location", CatalogUtils.URIToString(location)) :: - Row("Properties", "((a,a), (b,b), (c,c), (d,d))") :: Nil) } finally { catalog.reset() } } } - test("Alter Database - database does not exists") { - val databaseNames = Seq("db1", "`database`") - - databaseNames.foreach { dbName => - val dbNameWithoutBackTicks = cleanIdentifier(dbName) - assert(!spark.sessionState.catalog.databaseExists(dbNameWithoutBackTicks)) - - val message = intercept[AnalysisException] { - sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") - }.getMessage - assert(message.contains(s"Database '$dbNameWithoutBackTicks' not found")) - } - } - test("create table in default db") { val catalog = spark.sessionState.catalog val tableIdent1 = TableIdentifier("tab1", None) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceSetPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceSetPropertiesSuite.scala new file mode 100644 index 000000000000..cb89db3181f3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceSetPropertiesSuite.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 ... SET 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.AlterNamespaceSetPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterNamespaceSetPropertiesSuite` + */ +trait AlterNamespaceSetPropertiesSuiteBase extends command.AlterNamespaceSetPropertiesSuiteBase + with command.TestsV1AndV2Commands { + override def namespace: String = "db" + override def notFoundMsgPrefix: String = "Database" +} + +/** + * The class contains tests for the `ALTER NAMESPACE ... SET PROPERTIES` command to + * check V1 In-Memory table catalog. + */ +class AlterNamespaceSetPropertiesSuite extends AlterNamespaceSetPropertiesSuiteBase + with CommandSuiteBase { + override def commandVersion: String = super[AlterNamespaceSetPropertiesSuiteBase].commandVersion +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceSetPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceSetPropertiesSuite.scala new file mode 100644 index 000000000000..b5b352a7e7c4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceSetPropertiesSuite.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 ... SET PROPERTIES` command to check V2 table + * catalogs. + */ +class AlterNamespaceSetPropertiesSuite extends command.AlterNamespaceSetPropertiesSuiteBase + with CommandSuiteBase { + override def namespace: String = "ns1.ns2" + override def notFoundMsgPrefix: String = "Namespace" +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetPropertiesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetPropertiesSuite.scala new file mode 100644 index 000000000000..77dd071c4475 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetPropertiesSuite.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 ... SET PROPERTIES` command to check + * V1 Hive external table catalog. + */ +class AlterNamespaceSetPropertiesSuite extends v1.AlterNamespaceSetPropertiesSuiteBase + with CommandSuiteBase { + override def commandVersion: String = super[AlterNamespaceSetPropertiesSuiteBase].commandVersion +}