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 7bcc2b7f0c3d..44b2e20ee821 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 @@ -1193,15 +1193,6 @@ class DDLParserSuite extends AnalysisTest { "DESC TABLE COLUMN for a specific partition is not supported")) } - test("describe database") { - val sql1 = "DESCRIBE DATABASE EXTENDED a.b" - val sql2 = "DESCRIBE DATABASE a.b" - comparePlans(parsePlan(sql1), - DescribeNamespace(UnresolvedNamespace(Seq("a", "b")), extended = true)) - comparePlans(parsePlan(sql2), - DescribeNamespace(UnresolvedNamespace(Seq("a", "b")), extended = false)) - } - test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), DescribeRelation( 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 360c8beff436..7c4a5ea22b16 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 @@ -1233,26 +1233,6 @@ class DataSourceV2SQLSuite assert(exception.getMessage.contains("Namespace 'ns1' not found")) } - test("DescribeNamespace using v2 catalog") { - withNamespace("testcat.ns1.ns2") { - sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + - "'test namespace' LOCATION '/tmp/ns_test'") - val descriptionDf = sql("DESCRIBE NAMESPACE testcat.ns1.ns2") - assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === - Seq( - ("info_name", StringType), - ("info_value", StringType) - )) - val description = descriptionDf.collect() - assert(description === Seq( - Row("Namespace Name", "ns2"), - Row(SupportsNamespaces.PROP_COMMENT.capitalize, "test namespace"), - Row(SupportsNamespaces.PROP_LOCATION.capitalize, "/tmp/ns_test"), - Row(SupportsNamespaces.PROP_OWNER.capitalize, defaultUser)) - ) - } - } - test("ALTER NAMESPACE .. SET PROPERTIES using v2 catalog") { withNamespace("testcat.ns1.ns2") { sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " + @@ -2810,25 +2790,6 @@ class DataSourceV2SQLSuite } } - test("SPARK-34577: drop/add columns to a dataset of `DESCRIBE NAMESPACE`") { - withNamespace("ns") { - sql("CREATE NAMESPACE ns") - val description = sql(s"DESCRIBE NAMESPACE ns") - val noCommentDataset = description.drop("info_name") - val expectedSchema = new StructType() - .add( - name = "info_value", - dataType = StringType, - nullable = true, - metadata = new MetadataBuilder() - .putString("comment", "value of the namespace info").build()) - assert(noCommentDataset.schema === expectedSchema) - val isNullDataset = noCommentDataset - .withColumn("is_null", noCommentDataset("info_value").isNull) - assert(isNullDataset.schema === expectedSchema.add("is_null", BooleanType, false)) - } - } - test("SPARK-34923: do not propagate metadata columns through Project") { val t1 = s"${catalogAndNamespace}table" withTable(t1) { 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 ba87883b64e0..a3868666f979 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 @@ -757,14 +757,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"CREATE DATABASE $dbName") - checkAnswer( - sql(s"DESCRIBE DATABASE EXTENDED $dbName").toDF("key", "value") - .where("key not like 'Owner%'"), // filter for consistency with in-memory catalog - Row("Database Name", dbNameWithoutBackTicks) :: - Row("Comment", "") :: - Row("Location", CatalogUtils.URIToString(location)) :: - Row("Properties", "") :: Nil) - sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") checkAnswer( @@ -813,7 +805,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("Drop/Alter/Describe Database - database does not exists") { + test("Drop/Alter Database - database does not exists") { val databaseNames = Seq("db1", "`database`") databaseNames.foreach { dbName => @@ -829,13 +821,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')") }.getMessage assert(message.contains(s"Database '$dbNameWithoutBackTicks' not found")) - - message = intercept[AnalysisException] { - sql(s"DESCRIBE DATABASE EXTENDED $dbName") - }.getMessage - assert(message.contains(s"Database '$dbNameWithoutBackTicks' not found")) - - sql(s"DROP DATABASE IF EXISTS $dbName") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceParserSuite.scala new file mode 100644 index 000000000000..93da7aa0248f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceParserSuite.scala @@ -0,0 +1,39 @@ +/* + * 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.plans.logical.{DescribeNamespace, LogicalPlan} + +class DescribeNamespaceParserSuite extends AnalysisTest { + test("describe namespace") { + def check(sql: String, expected: LogicalPlan): Unit = { + comparePlans(parsePlan(sql), expected) + comparePlans(parsePlan(sql.replace("NAMESPACE", "DATABASE")), expected) + comparePlans(parsePlan(sql.replace("NAMESPACE", "SCHEMA")), expected) + } + + check("DESCRIBE NAMESPACE EXTENDED a.b", + DescribeNamespace(UnresolvedNamespace(Seq("a", "b")), extended = true)) + check("DESCRIBE NAMESPACE a.b", + DescribeNamespace(UnresolvedNamespace(Seq("a", "b")), extended = false)) + check("DESCRIBE NAMESPACE `a.b`", + DescribeNamespace(UnresolvedNamespace(Seq("a.b")), extended = false)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala new file mode 100644 index 000000000000..727bb90bb0cf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala @@ -0,0 +1,50 @@ +/* + * 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} + +/** + * This base suite contains unified tests for the `DESCRIBE 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.DescribeNamespaceSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.DescribeNamespaceSuiteBase` + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.DescribeNamespaceSuite` + * - V1 Hive External catalog: +* `org.apache.spark.sql.hive.execution.command.DescribeNamespaceSuite` + */ +trait DescribeNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "DESCRIBE NAMESPACE" + + protected def notFoundMsgPrefix: String + + test("namespace does not exists") { + val ns = "db1" + val message = intercept[AnalysisException] { + sql(s"DESCRIBE NAMESPACE EXTENDED $catalog.$ns") + }.getMessage + + assert(message.contains(s"$notFoundMsgPrefix '$ns' not found")) + + // TODO: Move this to DropNamespaceSuite when the test suite is introduced. + sql(s"DROP NAMESPACE IF EXISTS $catalog.$ns") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeNamespaceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeNamespaceSuite.scala new file mode 100644 index 000000000000..a86e4a51b052 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeNamespaceSuite.scala @@ -0,0 +1,59 @@ +/* + * 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.Row +import org.apache.spark.sql.execution.command + +/** + * This base suite contains unified tests for the `DESCRIBE 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.DescribeNamespaceSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.DescribeNamespaceSuite` + */ +trait DescribeNamespaceSuiteBase extends command.DescribeNamespaceSuiteBase { + override def notFoundMsgPrefix: String = "Database" + + test("basic") { + val ns = "db1" + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns") + + val result = sql(s"DESCRIBE NAMESPACE EXTENDED $ns") + .toDF("key", "value") + .where("key not like 'Owner%'") // filter for consistency with in-memory catalog + .collect() + + assert(result.length == 4) + assert(result(0) === Row("Database Name", ns)) + assert(result(1) === Row("Comment", "")) + // Check only the key for "Location" since its value depends on warehouse path, etc. + assert(result(2).getString(0) === "Location") + assert(result(3) === Row("Properties", "")) + } + } +} + +/** + * The class contains tests for the `DESCRIBE NAMESPACE` command to check V1 In-Memory + * table catalog. + */ +class DescribeNamespaceSuite extends DescribeNamespaceSuiteBase with CommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeNamespaceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeNamespaceSuite.scala new file mode 100644 index 000000000000..a98c6a486ae5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeNamespaceSuite.scala @@ -0,0 +1,70 @@ +/* + * 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.Row +import org.apache.spark.sql.connector.catalog.SupportsNamespaces +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.types.{BooleanType, MetadataBuilder, StringType, StructType} +import org.apache.spark.util.Utils + +/** + * The class contains tests for the `DESCRIBE NAMESPACE` command to check V2 table catalogs. + */ +class DescribeNamespaceSuite extends command.DescribeNamespaceSuiteBase with CommandSuiteBase { + override def notFoundMsgPrefix: String = "Namespace" + + test("DescribeNamespace using v2 catalog") { + withNamespace(s"$catalog.ns1.ns2") { + sql(s"CREATE NAMESPACE IF NOT EXISTS $catalog.ns1.ns2 COMMENT " + + "'test namespace' LOCATION '/tmp/ns_test'") + val descriptionDf = sql(s"DESCRIBE NAMESPACE $catalog.ns1.ns2") + assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === + Seq( + ("info_name", StringType), + ("info_value", StringType) + )) + val description = descriptionDf.collect() + assert(description === Seq( + Row("Namespace Name", "ns2"), + Row(SupportsNamespaces.PROP_COMMENT.capitalize, "test namespace"), + Row(SupportsNamespaces.PROP_LOCATION.capitalize, "/tmp/ns_test"), + Row(SupportsNamespaces.PROP_OWNER.capitalize, Utils.getCurrentUserName())) + ) + } + } + + test("SPARK-34577: drop/add columns to a dataset of `DESCRIBE NAMESPACE`") { + withNamespace("ns") { + sql("CREATE NAMESPACE ns") + val description = sql(s"DESCRIBE NAMESPACE ns") + val noCommentDataset = description.drop("info_name") + val expectedSchema = new StructType() + .add( + name = "info_value", + dataType = StringType, + nullable = true, + metadata = new MetadataBuilder() + .putString("comment", "value of the namespace info").build()) + assert(noCommentDataset.schema === expectedSchema) + val isNullDataset = noCommentDataset + .withColumn("is_null", noCommentDataset("info_value").isNull) + assert(isNullDataset.schema === expectedSchema.add("is_null", BooleanType, false)) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DescribeNamespaceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DescribeNamespaceSuite.scala new file mode 100644 index 000000000000..866cc178012d --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DescribeNamespaceSuite.scala @@ -0,0 +1,26 @@ +/* + * 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 `DESCRIBE NAMESPACE` command to check V1 Hive external + * table catalog. + */ +class DescribeNamespaceSuite extends v1.DescribeNamespaceSuiteBase with CommandSuiteBase