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 9354614c4121..babcf9d921ae 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 @@ -1686,90 +1686,6 @@ class DDLParserSuite extends AnalysisTest { ShowViews(UnresolvedNamespace(Seq("ns1")), Some("*test*"))) } - test("create namespace -- backward compatibility with DATABASE/DBPROPERTIES") { - val expected = CreateNamespace( - UnresolvedDBObjectName(Seq("a", "b", "c"), true), - ifNotExists = true, - Map( - "a" -> "a", - "b" -> "b", - "c" -> "c", - "comment" -> "namespace_comment", - "location" -> "/home/user/db")) - - comparePlans( - parsePlan( - """ - |CREATE NAMESPACE IF NOT EXISTS a.b.c - |WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c') - |COMMENT 'namespace_comment' LOCATION '/home/user/db' - """.stripMargin), - expected) - - comparePlans( - parsePlan( - """ - |CREATE DATABASE IF NOT EXISTS a.b.c - |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') - |COMMENT 'namespace_comment' LOCATION '/home/user/db' - """.stripMargin), - expected) - } - - test("create namespace -- check duplicates") { - def createDatabase(duplicateClause: String): String = { - s""" - |CREATE NAMESPACE IF NOT EXISTS a.b.c - |$duplicateClause - |$duplicateClause - """.stripMargin - } - val sql1 = createDatabase("COMMENT 'namespace_comment'") - val sql2 = createDatabase("LOCATION '/home/user/db'") - val sql3 = createDatabase("WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c')") - val sql4 = createDatabase("WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") - - intercept(sql1, "Found duplicate clauses: COMMENT") - intercept(sql2, "Found duplicate clauses: LOCATION") - intercept(sql3, "Found duplicate clauses: WITH PROPERTIES") - intercept(sql4, "Found duplicate clauses: WITH DBPROPERTIES") - } - - test("create namespace - property values must be set") { - assertUnsupported( - sql = "CREATE NAMESPACE a.b.c WITH PROPERTIES('key_without_value', 'key_with_value'='x')", - containsThesePhrases = Seq("key_without_value")) - } - - test("create namespace -- either PROPERTIES or DBPROPERTIES is allowed") { - val sql = - s""" - |CREATE NAMESPACE IF NOT EXISTS a.b.c - |WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c') - |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') - """.stripMargin - intercept(sql, "Either PROPERTIES or DBPROPERTIES is allowed") - } - - test("create namespace - support for other types in PROPERTIES") { - val sql = - """ - |CREATE NAMESPACE a.b.c - |LOCATION '/home/user/db' - |WITH PROPERTIES ('a'=1, 'b'=0.1, 'c'=TRUE) - """.stripMargin - comparePlans( - parsePlan(sql), - CreateNamespace( - UnresolvedDBObjectName(Seq("a", "b", "c"), true), - ifNotExists = false, - Map( - "a" -> "1", - "b" -> "0.1", - "c" -> "true", - "location" -> "/home/user/db"))) - } - 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 4166568f29af..93a0b1089f25 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 @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchNamespaceException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.connector.catalog._ @@ -1093,93 +1093,6 @@ class DataSourceV2SQLSuite " only SessionCatalog supports this command.")) } - test("CreateNameSpace: basic tests") { - // Session catalog is used. - withNamespace("ns") { - sql("CREATE NAMESPACE ns") - testShowNamespaces("SHOW NAMESPACES", Seq("default", "ns")) - } - - // V2 non-session catalog is used. - withNamespace("testcat.ns1.ns2") { - sql("CREATE NAMESPACE testcat.ns1.ns2") - testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1")) - testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns2")) - } - - withNamespace("testcat.test") { - withTempDir { tmpDir => - val path = tmpDir.getCanonicalPath - sql(s"CREATE NAMESPACE testcat.test LOCATION '$path'") - val metadata = - catalog("testcat").asNamespaceCatalog.loadNamespaceMetadata(Array("test")).asScala - val catalogPath = metadata(SupportsNamespaces.PROP_LOCATION) - assert(catalogPath.equals(catalogPath)) - } - } - } - - test("CreateNameSpace: test handling of 'IF NOT EXIST'") { - withNamespace("testcat.ns1") { - sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1") - - // The 'ns1' namespace already exists, so this should fail. - val exception = intercept[NamespaceAlreadyExistsException] { - sql("CREATE NAMESPACE testcat.ns1") - } - assert(exception.getMessage.contains("Namespace 'ns1' already exists")) - - // The following will be no-op since the namespace already exists. - sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1") - } - } - - test("CreateNameSpace: reserved properties") { - import SupportsNamespaces._ - withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { - CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => - val exception = intercept[ParseException] { - sql(s"CREATE NAMESPACE testcat.reservedTest WITH DBPROPERTIES('$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 WITH DBPROPERTIES('$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") - } - } - } - } - - test("SPARK-37456: Location in CreateNamespace should be qualified") { - withNamespace("testcat.ns1.ns2") { - val e = intercept[IllegalArgumentException] { - sql("CREATE NAMESPACE testcat.ns1.ns2 LOCATION ''") - } - assert(e.getMessage.contains("Can not create a Path from an empty string")) - - sql("CREATE NAMESPACE testcat.ns1.ns2 LOCATION '/tmp/ns_test'") - val descriptionDf = sql("DESCRIBE NAMESPACE EXTENDED testcat.ns1.ns2") - assert(descriptionDf.collect() === Seq( - Row("Namespace Name", "ns2"), - Row(SupportsNamespaces.PROP_LOCATION.capitalize, "file:/tmp/ns_test"), - Row(SupportsNamespaces.PROP_OWNER.capitalize, defaultUser), - Row("Properties", "")) - ) - } - } - test("create/replace/alter table - reserved properties") { import TableCatalog._ withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceParserSuite.scala new file mode 100644 index 000000000000..c3cb96814a50 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceParserSuite.scala @@ -0,0 +1,111 @@ +/* + * 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, UnresolvedDBObjectName} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.plans.logical.CreateNamespace + +class CreateNamespaceParserSuite extends AnalysisTest { + test("create namespace -- backward compatibility with DATABASE/DBPROPERTIES") { + val expected = CreateNamespace( + UnresolvedDBObjectName(Seq("a", "b", "c"), true), + ifNotExists = true, + Map( + "a" -> "a", + "b" -> "b", + "c" -> "c", + "comment" -> "namespace_comment", + "location" -> "/home/user/db")) + + comparePlans( + parsePlan( + """ + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c') + |COMMENT 'namespace_comment' LOCATION '/home/user/db' + """.stripMargin), + expected) + + comparePlans( + parsePlan( + """ + |CREATE DATABASE IF NOT EXISTS a.b.c + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + |COMMENT 'namespace_comment' LOCATION '/home/user/db' + """.stripMargin), + expected) + } + + test("create namespace -- check duplicates") { + def createNamespace(duplicateClause: String): String = { + s""" + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |$duplicateClause + |$duplicateClause + """.stripMargin + } + val sql1 = createNamespace("COMMENT 'namespace_comment'") + val sql2 = createNamespace("LOCATION '/home/user/db'") + val sql3 = createNamespace("WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c')") + val sql4 = createNamespace("WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + + intercept(sql1, "Found duplicate clauses: COMMENT") + intercept(sql2, "Found duplicate clauses: LOCATION") + intercept(sql3, "Found duplicate clauses: WITH PROPERTIES") + intercept(sql4, "Found duplicate clauses: WITH DBPROPERTIES") + } + + test("create namespace - property values must be set") { + intercept( + "CREATE NAMESPACE a.b.c WITH PROPERTIES('key_without_value', 'key_with_value'='x')", + "Operation not allowed: Values must be specified for key(s): [key_without_value]") + } + + test("create namespace -- either PROPERTIES or DBPROPERTIES is allowed") { + val sql = + s""" + |CREATE NAMESPACE IF NOT EXISTS a.b.c + |WITH PROPERTIES ('a'='a', 'b'='b', 'c'='c') + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + """.stripMargin + intercept(sql, "Either PROPERTIES or DBPROPERTIES is allowed") + } + + test("create namespace - support for other types in PROPERTIES") { + val sql = + """ + |CREATE NAMESPACE a.b.c + |LOCATION '/home/user/db' + |WITH PROPERTIES ('a'=1, 'b'=0.1, 'c'=TRUE) + """.stripMargin + comparePlans( + parsePlan(sql), + CreateNamespace( + UnresolvedDBObjectName(Seq("a", "b", "c"), true), + ifNotExists = false, + Map( + "a" -> "1", + "b" -> "0.1", + "c" -> "true", + "location" -> "/home/user/db"))) + } + + private def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(parsePlan)(sqlCommand, messages: _*) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala new file mode 100644 index 000000000000..597ea869eb8a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateNamespaceSuiteBase.scala @@ -0,0 +1,141 @@ +/* + * 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 scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, CatalogV2Util, SupportsNamespaces} +import org.apache.spark.sql.internal.SQLConf + +/** + * This base suite contains unified tests for the `CREATE NAMESPACE` 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.CreateNamespaceSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.CreateNamespaceSuiteBase` + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.CreateNamespaceSuite` + * - V1 Hive External catalog: +* `org.apache.spark.sql.hive.execution.command.CreateNamespaceSuite` + */ +trait CreateNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override val command = "CREATE NAMESPACE" + + protected def namespace: String + + protected def namespaceArray: Array[String] = namespace.split('.') + + protected def notFoundMsgPrefix: String + + protected def alreadyExistErrorMessage: String = s"$notFoundMsgPrefix '$namespace' already exists" + + test("basic") { + val ns = s"$catalog.$namespace" + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns") + assert(getCatalog(catalog).asNamespaceCatalog.namespaceExists(namespaceArray)) + } + } + + test("namespace with location") { + val ns = s"$catalog.$namespace" + withNamespace(ns) { + withTempDir { tmpDir => + // The generated temp path is not qualified. + val path = tmpDir.getCanonicalPath + assert(!path.startsWith("file:/")) + + val e = intercept[IllegalArgumentException] { + sql(s"CREATE NAMESPACE $ns LOCATION ''") + } + assert(e.getMessage.contains("Can not create a Path from an empty string")) + + val uri = new Path(path).toUri + sql(s"CREATE NAMESPACE $ns LOCATION '$uri'") + + // Make sure the location is qualified. + val expected = makeQualifiedPath(tmpDir.toString) + assert("file" === expected.getScheme) + assert(new Path(getNamespaceLocation(catalog, namespaceArray)).toUri === expected) + } + } + } + + test("Namespace already exists") { + val ns = s"$catalog.$namespace" + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns") + + // TODO: HiveExternalCatalog throws DatabaseAlreadyExistsException, and + // non-Hive catalogs throw NamespaceAlreadyExistsException. + val e = intercept[AnalysisException] { + sql(s"CREATE NAMESPACE $ns") + } + assert(e.getMessage.contains(alreadyExistErrorMessage)) + + // The following will be no-op since the namespace already exists. + sql(s"CREATE NAMESPACE IF NOT EXISTS $ns") + } + } + + test("CreateNameSpace: reserved properties") { + import SupportsNamespaces._ + val ns = s"$catalog.$namespace" + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + val exception = intercept[ParseException] { + sql(s"CREATE NAMESPACE $ns WITH DBPROPERTIES('$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) { + sql(s"CREATE NAMESPACE $ns WITH DBPROPERTIES('$key'='foo')") + assert(sql(s"DESC NAMESPACE EXTENDED $ns") + .toDF("k", "v") + .where("k='Properties'") + .where("v=''") + .count == 1, s"$key is a reserved namespace property and ignored") + val meta = + getCatalog(catalog).asNamespaceCatalog.loadNamespaceMetadata(namespaceArray) + assert(meta.get(key) == null || !meta.get(key).contains("foo"), + "reserved properties should not have side effects") + } + } + } + } + + protected def getNamespaceLocation(catalog: String, namespace: Array[String]): String = { + val metadata = getCatalog(catalog).asNamespaceCatalog + .loadNamespaceMetadata(namespace).asScala + metadata(SupportsNamespaces.PROP_LOCATION) + } + + private def getCatalog(name: String): CatalogPlugin = { + spark.sessionState.catalogManager.catalog(name) + } +} 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 56de04591992..4380618de988 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 @@ -333,97 +333,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { testChangeColumn(isDatasourceTable = true) } - test("the qualified path of a database is stored in the catalog") { - val catalog = spark.sessionState.catalog - - withTempDir { tmpDir => - val path = tmpDir.getCanonicalPath - // The generated temp path is not qualified. - assert(!path.startsWith("file:/")) - val uri = tmpDir.toURI - sql(s"CREATE DATABASE db1 LOCATION '$uri'") - val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri - assert("file" === pathInCatalog.getScheme) - val expectedPath = new Path(path).toUri - assert(expectedPath.getPath === pathInCatalog.getPath) - sql("DROP DATABASE db1") - } - } - - test("Create Database using Default Warehouse Path") { - val catalog = spark.sessionState.catalog - val dbName = "db1" - try { - sql(s"CREATE DATABASE $dbName") - val db1 = catalog.getDatabaseMetadata(dbName) - assert(db1.copy(properties = db1.properties -- reversedProperties) == CatalogDatabase( - dbName, - "", - getDBPath(dbName), - Map.empty)) - sql(s"DROP DATABASE $dbName CASCADE") - assert(!catalog.databaseExists(dbName)) - } finally { - catalog.reset() - } - } - - test("Create/Drop Database - location") { - val catalog = spark.sessionState.catalog - val databaseNames = Seq("db1", "`database`") - withTempDir { tmpDir => - val path = new Path(tmpDir.getCanonicalPath).toUri - databaseNames.foreach { dbName => - try { - val e = intercept[IllegalArgumentException] { - sql(s"CREATE DATABASE $dbName Location ''") - } - assert(e.getMessage.contains("Can not create a Path from an empty string")) - - val dbNameWithoutBackTicks = cleanIdentifier(dbName) - sql(s"CREATE DATABASE $dbName Location '$path'") - val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expPath = makeQualifiedPath(tmpDir.toString) - assert(db1.copy(properties = db1.properties -- reversedProperties) == CatalogDatabase( - dbNameWithoutBackTicks, - "", - expPath, - Map.empty)) - sql(s"DROP DATABASE $dbName CASCADE") - assert(!catalog.databaseExists(dbNameWithoutBackTicks)) - } finally { - catalog.reset() - } - } - } - } - - test("Create Database - database already exists") { - val catalog = spark.sessionState.catalog - val databaseNames = Seq("db1", "`database`") - - databaseNames.foreach { dbName => - try { - val dbNameWithoutBackTicks = cleanIdentifier(dbName) - sql(s"CREATE DATABASE $dbName") - val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - assert(db1.copy(properties = db1.properties -- reversedProperties) == CatalogDatabase( - dbNameWithoutBackTicks, - "", - getDBPath(dbNameWithoutBackTicks), - Map.empty)) - - // TODO: HiveExternalCatalog should throw DatabaseAlreadyExistsException - val e = intercept[AnalysisException] { - sql(s"CREATE DATABASE $dbName") - }.getMessage - assert(e.contains(s"already exists")) - } finally { - catalog.reset() - } - } - } - private def withEmptyDirInTablePath(dirName: String)(f : File => Unit): Unit = { val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier(dirName))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateNamespaceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateNamespaceSuite.scala new file mode 100644 index 000000000000..ba3eba932355 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/CreateNamespaceSuite.scala @@ -0,0 +1,62 @@ +/* + * 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 java.net.URI + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.catalyst.catalog.CatalogUtils +import org.apache.spark.sql.execution.command + +/** + * This base suite contains unified tests for the `CREATE NAMESPACE` 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.CreateNamespaceSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.CreateNamespaceSuite` + */ +trait CreateNamespaceSuiteBase extends command.CreateNamespaceSuiteBase + with command.TestsV1AndV2Commands { + override def namespace: String = "db" + override def notFoundMsgPrefix: String = "Database" + + test("Create namespace using default warehouse path") { + val ns = s"$catalog.$namespace" + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns") + assert(makeQualifiedPath(getNamespaceLocation(catalog, namespaceArray)) + === getDBPath(namespace)) + } + } + + private def getDBPath(dbName: String): URI = { + val warehousePath = makeQualifiedPath(spark.sessionState.conf.warehousePath) + new Path(CatalogUtils.URIToString(warehousePath), s"$dbName.db").toUri + } +} + +/** + * The class contains tests for the `CREATE NAMESPACE` command to check V1 In-Memory + * table catalog. + */ +class CreateNamespaceSuite extends CreateNamespaceSuiteBase with CommandSuiteBase { + override def commandVersion: String = super[CreateNamespaceSuiteBase].commandVersion +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateNamespaceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateNamespaceSuite.scala new file mode 100644 index 000000000000..66865ff0c5bd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/CreateNamespaceSuite.scala @@ -0,0 +1,28 @@ +/* + * 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 `CREATE NAMESPACE` command to check V2 table catalogs. + */ +class CreateNamespaceSuite extends command.CreateNamespaceSuiteBase 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/CreateNamespaceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CreateNamespaceSuite.scala new file mode 100644 index 000000000000..e8f85f40e9fb --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CreateNamespaceSuite.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 `CREATE NAMESPACE` command to check V1 Hive external + * table catalog. + */ +class CreateNamespaceSuite extends v1.CreateNamespaceSuiteBase with CommandSuiteBase { + override def commandVersion: String = super[CreateNamespaceSuiteBase].commandVersion + override def alreadyExistErrorMessage: String = s"$notFoundMsgPrefix $namespace already exists" +}