Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 " +
Expand Down Expand Up @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down Expand Up @@ -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 =>
Expand All @@ -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")
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -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))
}
}
Original file line number Diff line number Diff line change
@@ -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")
Copy link
Contributor

Choose a reason for hiding this comment

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

We don't need to test DROP NAMESPACE here

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is testing whether IF EXISTS is respected when the namespace doesn't exist (existing test). Do we still want to remove it?

Copy link
Contributor

Choose a reason for hiding this comment

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

Well, it should belong to DropNamespaceSuite if we have one.

Copy link
Contributor

Choose a reason for hiding this comment

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

OK seems we don't have existing tests cover this case. Let's add it back with a TODO to move it to DropNameSpaceSuite in the future when we have one.

}
}
Original file line number Diff line number Diff line change
@@ -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
Original file line number Diff line number Diff line change
@@ -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))
}
}
}
Original file line number Diff line number Diff line change
@@ -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