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 @@ -1770,33 +1770,6 @@ class DDLParserSuite extends AnalysisTest {
"location" -> "/home/user/db")))
}

test("drop namespace") {
comparePlans(
parsePlan("DROP NAMESPACE a.b.c"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = false, cascade = false))

comparePlans(
parsePlan("DROP NAMESPACE IF EXISTS a.b.c"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = true, cascade = false))

comparePlans(
parsePlan("DROP NAMESPACE IF EXISTS a.b.c RESTRICT"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = true, cascade = false))

comparePlans(
parsePlan("DROP NAMESPACE IF EXISTS a.b.c CASCADE"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = true, cascade = true))

comparePlans(
parsePlan("DROP NAMESPACE a.b.c CASCADE"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = false, cascade = true))
}

test("set namespace properties") {
comparePlans(
parsePlan("ALTER DATABASE a.b.c SET PROPERTIES ('a'='a', 'b'='b', 'c'='c')"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import java.time.{Duration, LocalDate, Period}

import scala.collection.JavaConverters._

import org.apache.spark.SparkException
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, TableAlreadyExistsException}
Expand Down Expand Up @@ -1192,69 +1191,6 @@ class DataSourceV2SQLSuite
}
}

test("DropNamespace: basic tests") {
// Session catalog is used.
sql("CREATE NAMESPACE ns")
testShowNamespaces("SHOW NAMESPACES", Seq("default", "ns"))
sql("DROP NAMESPACE ns")
testShowNamespaces("SHOW NAMESPACES", Seq("default"))

// V2 non-session catalog is used.
sql("CREATE NAMESPACE testcat.ns1")
testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1"))
sql("DROP NAMESPACE testcat.ns1")
testShowNamespaces("SHOW NAMESPACES IN testcat", Seq())
}

test("DropNamespace: drop non-empty namespace with a non-cascading mode") {
sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo")
sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo")
testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1"))
testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns2"))

def assertDropFails(): Unit = {
val e = intercept[SparkException] {
sql("DROP NAMESPACE testcat.ns1")
}
assert(e.getMessage.contains("Cannot drop a non-empty namespace: ns1"))
}

// testcat.ns1.table is present, thus testcat.ns1 cannot be dropped.
assertDropFails()
sql("DROP TABLE testcat.ns1.table")

// testcat.ns1.ns2.table is present, thus testcat.ns1 cannot be dropped.
assertDropFails()
sql("DROP TABLE testcat.ns1.ns2.table")

// testcat.ns1.ns2 namespace is present, thus testcat.ns1 cannot be dropped.
assertDropFails()
sql("DROP NAMESPACE testcat.ns1.ns2")

// Now that testcat.ns1 is empty, it can be dropped.
sql("DROP NAMESPACE testcat.ns1")
testShowNamespaces("SHOW NAMESPACES IN testcat", Seq())
}

test("DropNamespace: drop non-empty namespace with a cascade mode") {
sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo")
sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo")
testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1"))
testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns2"))

sql("DROP NAMESPACE testcat.ns1 CASCADE")
testShowNamespaces("SHOW NAMESPACES IN testcat", Seq())
}

test("DropNamespace: test handling of 'IF EXISTS'") {
sql("DROP NAMESPACE IF EXISTS testcat.unknown")

val exception = intercept[NoSuchNamespaceException] {
sql("DROP NAMESPACE testcat.ns1")
}
assert(exception.getMessage.contains("Namespace 'ns1' not found"))
}

test("ALTER NAMESPACE .. SET PROPERTIES using v2 catalog") {
withNamespace("testcat.ns1.ns2") {
sql("CREATE NAMESPACE IF NOT EXISTS testcat.ns1.ns2 COMMENT " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -787,64 +787,20 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
}
}

test("Drop/Alter Database - database does not exists") {
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))

var message = intercept[AnalysisException] {
sql(s"DROP DATABASE $dbName")
}.getMessage
assert(message.contains(s"Database '$dbNameWithoutBackTicks' not found"))

message = intercept[AnalysisException] {
val message = intercept[AnalysisException] {
sql(s"ALTER DATABASE $dbName SET DBPROPERTIES ('d'='d')")
}.getMessage
assert(message.contains(s"Database '$dbNameWithoutBackTicks' not found"))
}
}

test("drop non-empty database in restrict mode") {
val catalog = spark.sessionState.catalog
val dbName = "db1"
sql(s"CREATE DATABASE $dbName")

// create a table in database
val tableIdent1 = TableIdentifier("tab1", Some(dbName))
createTable(catalog, tableIdent1)

// drop a non-empty database in Restrict mode
val message = intercept[AnalysisException] {
sql(s"DROP DATABASE $dbName RESTRICT")
}.getMessage
assert(message.contains(s"Database $dbName is not empty. One or more tables exist"))


catalog.dropTable(tableIdent1, ignoreIfNotExists = false, purge = false)

assert(catalog.listDatabases().contains(dbName))
sql(s"DROP DATABASE $dbName RESTRICT")
assert(!catalog.listDatabases().contains(dbName))
}

test("drop non-empty database in cascade mode") {
val catalog = spark.sessionState.catalog
val dbName = "db1"
sql(s"CREATE DATABASE $dbName")

// create a table in database
val tableIdent1 = TableIdentifier("tab1", Some(dbName))
createTable(catalog, tableIdent1)

// drop a non-empty database in CASCADE mode
assert(catalog.listTables(dbName).contains(tableIdent1))
assert(catalog.listDatabases().contains(dbName))
sql(s"DROP DATABASE $dbName CASCADE")
assert(!catalog.listDatabases().contains(dbName))
}

test("create table in default db") {
val catalog = spark.sessionState.catalog
val tableIdent1 = TableIdentifier("tab1", None)
Expand Down Expand Up @@ -1541,39 +1497,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
}
}

test("drop current database") {
withDatabase("temp") {
sql("CREATE DATABASE temp")
sql("USE temp")
sql("DROP DATABASE temp")
val e = intercept[AnalysisException] {
sql("CREATE TABLE t (a INT, b INT) USING parquet")
}.getMessage
assert(e.contains("Database 'temp' not found"))
}
}

test("drop default database") {
val caseSensitiveOptions = if (isUsingHiveMetastore) Seq("false") else Seq("true", "false")
caseSensitiveOptions.foreach { caseSensitive =>
withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive) {
var message = intercept[AnalysisException] {
sql("DROP DATABASE default")
}.getMessage
assert(message.contains("Can not drop default database"))

message = intercept[AnalysisException] {
sql("DROP DATABASE DeFault")
}.getMessage
if (caseSensitive == "true") {
assert(message.contains("Database 'DeFault' not found"))
} else {
assert(message.contains("Can not drop default database"))
}
}
}
}

test("create temporary view with mismatched schema") {
withTable("tab1") {
spark.range(10).write.saveAsTable("tab1")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,6 @@ trait DescribeNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils {
}.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")
}

test("Keep the legacy output schema") {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.DropNamespace

class DropNamespaceParserSuite extends AnalysisTest {
test("drop namespace") {
comparePlans(
parsePlan("DROP NAMESPACE a.b.c"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = false, cascade = false))

comparePlans(
parsePlan("DROP NAMESPACE IF EXISTS a.b.c"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = true, cascade = false))

comparePlans(
parsePlan("DROP NAMESPACE IF EXISTS a.b.c RESTRICT"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = true, cascade = false))

comparePlans(
parsePlan("DROP NAMESPACE IF EXISTS a.b.c CASCADE"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = true, cascade = true))

comparePlans(
parsePlan("DROP NAMESPACE a.b.c CASCADE"),
DropNamespace(
UnresolvedNamespace(Seq("a", "b", "c")), ifExists = false, cascade = true))
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
/*
* 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, Row}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StringType, StructType}

/**
* This base suite contains unified tests for the `DROP 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.DropNamespaceSuite`
* - V1 table catalog tests: `org.apache.spark.sql.execution.command.v1.DropNamespaceSuiteBase`
* - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.DropNamespaceSuite`
* - V1 Hive External catalog: `org.apache.spark.sql.hive.execution.command.DropNamespaceSuite`
*/
trait DropNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils {
override val command = "DROP NAMESPACE"

protected def builtinTopNamespaces: Seq[String] = Seq.empty
protected def isCasePreserving: Boolean = true
protected def assertDropFails(): Unit

protected def checkNamespace(expected: Seq[String]) = {
val df = spark.sql(s"SHOW NAMESPACES IN $catalog")
assert(df.schema === new StructType().add("namespace", StringType, false))
checkAnswer(df, expected.map(Row(_)))
}

test("basic tests") {
sql(s"CREATE NAMESPACE $catalog.ns")
checkNamespace(Seq("ns") ++ builtinTopNamespaces)

sql(s"DROP NAMESPACE $catalog.ns")
checkNamespace(builtinTopNamespaces)
}

test("test handling of 'IF EXISTS'") {
// It must not throw any exceptions
sql(s"DROP NAMESPACE IF EXISTS $catalog.unknown")
checkNamespace(builtinTopNamespaces)
}

test("namespace does not exist") {
// Namespace $catalog.unknown does not exist.
val message = intercept[AnalysisException] {
sql(s"DROP NAMESPACE $catalog.unknown")
}.getMessage
assert(message.contains(s"'unknown' not found"))
}

test("drop non-empty namespace with a non-cascading mode") {
sql(s"CREATE NAMESPACE $catalog.ns")
sql(s"CREATE TABLE $catalog.ns.table (id bigint) $defaultUsing")
checkNamespace(Seq("ns") ++ builtinTopNamespaces)

// $catalog.ns.table is present, thus $catalog.ns cannot be dropped.
assertDropFails()
sql(s"DROP TABLE $catalog.ns.table")

// Now that $catalog.ns is empty, it can be dropped.
sql(s"DROP NAMESPACE $catalog.ns")
checkNamespace(builtinTopNamespaces)
}

test("drop non-empty namespace with a cascade mode") {
sql(s"CREATE NAMESPACE $catalog.ns")
sql(s"CREATE TABLE $catalog.ns.table (id bigint) $defaultUsing")
checkNamespace(Seq("ns") ++ builtinTopNamespaces)

sql(s"DROP NAMESPACE $catalog.ns CASCADE")
checkNamespace(builtinTopNamespaces)
}

test("drop current namespace") {
sql(s"CREATE NAMESPACE $catalog.ns")
sql(s"USE $catalog.ns")
sql(s"DROP NAMESPACE $catalog.ns")
checkNamespace(builtinTopNamespaces)
}

test("drop namespace with case sensitivity") {
Seq(true, false).foreach { caseSensitive =>
withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) {
sql(s"CREATE NAMESPACE $catalog.AAA")
sql(s"CREATE NAMESPACE $catalog.bbb")
// TODO: The v1 in-memory catalog should be case preserving as well.
val casePreserving = isCasePreserving && (catalogVersion == "V2" || caseSensitive)
val expected = if (casePreserving) "AAA" else "aaa"

sql(s"DROP NAMESPACE $catalog.$expected")
sql(s"DROP NAMESPACE $catalog.bbb")
checkNamespace(builtinTopNamespaces)
}
}
}
}
Loading