From db8968a2a3a9b88dc2d22fbf13fba074c6372681 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 2 Oct 2019 16:37:15 -0700 Subject: [PATCH 1/5] initial checkin --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 +- .../sql/catalyst/parser/AstBuilder.scala | 4 ++ .../sql/catalyst/parser/DDLParserSuite.scala | 19 ++++++ .../catalyst/parser/ParserUtilsSuite.scala | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 11 ---- .../sql/execution/command/databases.scala | 62 ------------------- .../datasources/DataSourceResolution.scala | 17 +---- .../sql/connector/DataSourceV2SQLSuite.scala | 27 +++++--- .../execution/command/DDLParserSuite.scala | 11 ---- 9 files changed, 48 insertions(+), 108 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index aa96e079070f..d831b1d4b69b 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -93,8 +93,7 @@ statement SET locationSpec #setDatabaseLocation | DROP database (IF EXISTS)? db=errorCapturingIdentifier (RESTRICT | CASCADE)? #dropDatabase - | SHOW DATABASES (LIKE? pattern=STRING)? #showDatabases - | SHOW NAMESPACES ((FROM | IN) multipartIdentifier)? + | SHOW (DATABASES | NAMESPACES) ((FROM | IN) multipartIdentifier)? (LIKE? pattern=STRING)? #showNamespaces | createTableHeader ('(' colTypeList ')')? tableProvider ((OPTIONS options=tablePropertyList) | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index f3c603abb9a0..a731b994dd5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -2304,6 +2304,10 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a [[ShowNamespacesStatement]] command. */ override def visitShowNamespaces(ctx: ShowNamespacesContext): LogicalPlan = withOrigin(ctx) { + if (ctx.DATABASES != null && ctx.multipartIdentifier != null) { + throw new ParseException(s"FROM/IN operator is not allowed in SHOW DATABASES", ctx) + } + ShowNamespacesStatement( Option(ctx.multipartIdentifier).map(visitMultipartIdentifier), Option(ctx.pattern).map(string)) 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 c0d444e5f5da..df1bbe9554f4 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 @@ -846,6 +846,25 @@ class DDLParserSuite extends AnalysisTest { ShowTablesStatement(Some(Seq("tbl")), Some("*dog*"))) } + test("show databases: basic") { + comparePlans( + parsePlan("SHOW DATABASES"), + ShowNamespacesStatement(None, None)) + comparePlans( + parsePlan("SHOW DATABASES LIKE 'defau*'"), + ShowNamespacesStatement(None, Some("defau*"))) + } + + test("show databases: FROM/IN operator is not allowed") { + def verify(sql: String): Unit = { + val exc = intercept[ParseException] { parsePlan(sql) } + assert(exc.getMessage.contains("FROM/IN operator is not allowed in SHOW DATABASES")) + } + + verify("SHOW DATABASES FROM testcat.ns1.ns2") + verify("SHOW DATABASES IN testcat.ns1.ns2") + } + test("show namespaces") { comparePlans( parsePlan("SHOW NAMESPACES"), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala index cba24fbe2a65..07f77ea889db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala @@ -40,7 +40,7 @@ class ParserUtilsSuite extends SparkFunSuite { } val showDbsContext = buildContext("show databases like 'identifier_with_wildcards'") { parser => - parser.statement().asInstanceOf[ShowDatabasesContext] + parser.statement().asInstanceOf[ShowNamespacesContext] } val createDbContext = buildContext( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 2346545876b2..3dd392156d52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -155,17 +155,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { partitionSpec = partitionSpec) } - /** - * Create a [[ShowDatabasesCommand]] logical plan. - * Example SQL: - * {{{ - * SHOW (DATABASES|SCHEMAS) [LIKE 'identifier_with_wildcards']; - * }}} - */ - override def visitShowDatabases(ctx: ShowDatabasesContext): LogicalPlan = withOrigin(ctx) { - ShowDatabasesCommand(Option(ctx.pattern).map(string)) - } - /** * A command for users to list the properties for a table. If propertyKey is specified, the value * for the propertyKey is returned. If propertyKey is not specified, all the keys and their diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala deleted file mode 100644 index 470c736da98b..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.{Row, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.types.StringType - - -/** - * A command for users to list the databases/schemas. - * If a databasePattern is supplied then the databases that only match the - * pattern would be listed. - * The syntax of using this command in SQL is: - * {{{ - * SHOW (DATABASES|SCHEMAS) [LIKE 'identifier_with_wildcards']; - * }}} - */ -case class ShowDatabasesCommand(databasePattern: Option[String]) extends RunnableCommand { - - // The result of SHOW DATABASES has one column called 'databaseName' - override val output: Seq[Attribute] = { - AttributeReference("databaseName", StringType, nullable = false)() :: Nil - } - - override def run(sparkSession: SparkSession): Seq[Row] = { - val catalog = sparkSession.sessionState.catalog - val databases = - databasePattern.map(catalog.listDatabases).getOrElse(catalog.listDatabases()) - databases.map { d => Row(d) } - } -} - - -/** - * Command for setting the current database. - * {{{ - * USE database_name; - * }}} - */ -case class SetDatabaseCommand(databaseName: String) extends RunnableCommand { - - override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.sessionState.catalog.setCurrentDatabase(databaseName) - Seq.empty[Row] - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index 33e1ecd359ad..1b299e560dc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -170,22 +170,11 @@ case class DataSourceResolution( DeleteFromTable(aliased, delete.condition) case ShowNamespacesStatement(None, pattern) => - defaultCatalog match { - case Some(catalog) => - ShowNamespaces(catalog.asNamespaceCatalog, None, pattern) - case None => - throw new AnalysisException("No default v2 catalog is set.") - } + ShowNamespaces(currentCatalog.asNamespaceCatalog, None, pattern) case ShowNamespacesStatement(Some(namespace), pattern) => - val DefaultCatalogAndNamespace(maybeCatalog, ns) = namespace - maybeCatalog match { - case Some(catalog) => - ShowNamespaces(catalog.asNamespaceCatalog, Some(ns), pattern) - case None => - throw new AnalysisException( - s"No v2 catalog is available for ${namespace.quoted}") - } + val CurrentCatalogAndNamespace(catalog, ns) = namespace + ShowNamespaces(catalog.asNamespaceCatalog, Some(ns), pattern) case update: UpdateTableStatement => throw new AnalysisException(s"Update table is not supported temporarily.") 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 ff3d828ff312..842c78509e17 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 @@ -780,11 +780,8 @@ class DataSourceV2SQLSuite test("ShowNamespaces: default v2 catalog is not set") { spark.sql("CREATE TABLE testcat.ns.table (id bigint) USING foo") - val exception = intercept[AnalysisException] { - sql("SHOW NAMESPACES") - } - - assert(exception.getMessage.contains("No default v2 catalog is set")) + // The current catalog is resolved to a v2 session catalog. + testShowNamespaces("SHOW NAMESPACES", Seq("default")) } test("ShowNamespaces: default v2 catalog doesn't support namespace") { @@ -812,12 +809,28 @@ class DataSourceV2SQLSuite assert(exception.getMessage.contains("does not support namespaces")) } - test("ShowNamespaces: no v2 catalog is available") { + test("ShowNamespaces: session catalog is used and namespace doesn't exist") { val exception = intercept[AnalysisException] { sql("SHOW NAMESPACES in dummy") } - assert(exception.getMessage.contains("No v2 catalog is available")) + assert(exception.getMessage.contains("Namespace 'dummy' not found")) + } + + test("ShowNamespaces: change catalog and namespace with USE statements") { + sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo") + + // Initially, the current catalog is a v2 session catalog. + testShowNamespaces("SHOW NAMESPACES", Seq("default")) + + // Update the current catalog to 'testcat'. + sql("USE testcat") + testShowNamespaces("SHOW NAMESPACES", Seq("ns1")) + + // Update the current namespace to 'ns1'. + sql("USE ns1") + // 'SHOW NAMESPACES' is not affected by the current namespace and lists root namespaces. + testShowNamespaces("SHOW NAMESPACES", Seq("ns1")) } private def testShowNamespaces( 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 c31ae1e4d83e..303630d9d0cb 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 @@ -811,17 +811,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { """.stripMargin) } - test("show databases") { - val sql1 = "SHOW DATABASES" - val sql2 = "SHOW DATABASES LIKE 'defau*'" - val parsed1 = parser.parsePlan(sql1) - val expected1 = ShowDatabasesCommand(None) - val parsed2 = parser.parsePlan(sql2) - val expected2 = ShowDatabasesCommand(Some("defau*")) - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - } - test("show tblproperties") { val parsed1 = parser.parsePlan("SHOW TBLPROPERTIES tab1") val expected1 = ShowTablePropertiesCommand(TableIdentifier("tab1", None), None) From 67114a70b327a74028b7aa47f9998392fce3b03a Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 2 Oct 2019 20:37:22 -0700 Subject: [PATCH 2/5] Fix unit tests --- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 0a9408ff6d5e..8bf89e0f93d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2691,7 +2691,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { sparkContext.addSparkListener(listener) try { // Execute the command. - sql("show databases").head() + sql("EXPLAIN show databases").head() // Make sure we have seen all events triggered by DataFrame.show() sparkContext.listenerBus.waitUntilEmpty() From 51966bccf0634e44ccaee885daa3a2de87bf5a60 Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Fri, 4 Oct 2019 10:36:32 -0700 Subject: [PATCH 3/5] Merge with [SPARK-29039][SQL] centralize the catalog and table lookup logic --- .../spark/sql/catalyst/analysis/ResolveCatalogs.scala | 11 ++++------- .../spark/sql/connector/catalog/LookupCatalog.scala | 7 ++++--- .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 9 --------- 3 files changed, 8 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 3757569443e7..5e5d15cdf67e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -165,13 +165,11 @@ class ResolveCatalogs(val catalogManager: CatalogManager) s"Can not specify catalog `${catalog.name}` for view ${viewName.quoted} " + s"because view support in catalog has not been implemented yet") - case ShowNamespacesStatement(Some(NonSessionCatalog(catalog, nameParts)), pattern) => - val namespace = if (nameParts.isEmpty) None else Some(nameParts) + case ShowNamespacesStatement(Some(CurrentCatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) - // TODO (SPARK-29014): we should check if the current catalog is not session catalog here. - case ShowNamespacesStatement(None, pattern) if defaultCatalog.isDefined => - ShowNamespaces(defaultCatalog.get.asNamespaceCatalog, None, pattern) + case ShowNamespacesStatement(None, pattern) => + ShowNamespaces(currentCatalog.asNamespaceCatalog, None, pattern) case ShowTablesStatement(Some(NonSessionCatalog(catalog, nameParts)), pattern) => ShowTables(catalog.asTableCatalog, nameParts, pattern) @@ -185,8 +183,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) SetCatalogAndNamespace(catalogManager, None, Some(nameParts)) } else { val CurrentCatalogAndNamespace(catalog, namespace) = nameParts - val ns = if (namespace.isEmpty) { None } else { Some(namespace) } - SetCatalogAndNamespace(catalogManager, Some(catalog.name()), ns) + SetCatalogAndNamespace(catalogManager, Some(catalog.name()), namespace) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index be4a7eba5910..1b5197f31842 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -102,7 +102,7 @@ private[sql] trait LookupCatalog extends Logging { } } - type CurrentCatalogAndNamespace = (CatalogPlugin, Seq[String]) + type CurrentCatalogAndNamespace = (CatalogPlugin, Option[Seq[String]]) /** * Extract catalog and namespace from a multi-part identifier with the current catalog if needed. @@ -112,10 +112,11 @@ private[sql] trait LookupCatalog extends Logging { def unapply(parts: Seq[String]): Some[CurrentCatalogAndNamespace] = parts match { case Seq(catalogName, tail @ _*) => try { - Some((catalogManager.catalog(catalogName), tail)) + Some( + (catalogManager.catalog(catalogName), if (tail.isEmpty) { None } else { Some(tail) })) } catch { case _: CatalogNotFoundException => - Some((currentCatalog, parts)) + Some((currentCatalog, Some(parts))) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 1f3b4319c0ea..e23b617af48b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -256,15 +256,6 @@ class ResolveSessionCatalog( case DropViewStatement(SessionCatalog(catalog, viewName), ifExists) => DropTableCommand(viewName.asTableIdentifier, ifExists, isView = true, purge = false) - case ShowNamespacesStatement(Some(SessionCatalog(catalog, nameParts)), pattern) => - throw new AnalysisException( - "SHOW NAMESPACES is not supported with the session catalog.") - - // TODO (SPARK-29014): we should check if the current catalog is session catalog here. - case ShowNamespacesStatement(None, pattern) if defaultCatalog.isEmpty => - throw new AnalysisException( - "SHOW NAMESPACES is not supported with the session catalog.") - case ShowTablesStatement(Some(SessionCatalog(catalog, nameParts)), pattern) => if (nameParts.length != 1) { throw new AnalysisException( From 94411cb730953cc06d28aef756922492d597414c Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Tue, 8 Oct 2019 21:26:05 -0700 Subject: [PATCH 4/5] Address PR comments --- .../catalyst/analysis/ResolveCatalogs.scala | 4 ++-- .../sql/connector/catalog/LookupCatalog.scala | 24 +++---------------- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- 3 files changed, 6 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 3a90156bc3c9..981ef1645940 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -169,7 +169,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) s"Can not specify catalog `${catalog.name}` for view ${viewName.quoted} " + s"because view support in catalog has not been implemented yet") - case ShowNamespacesStatement(Some(CurrentCatalogAndNamespace(catalog, namespace)), pattern) => + case ShowNamespacesStatement(Some(CatalogAndNamespace(catalog, namespace)), pattern) => ShowNamespaces(catalog.asNamespaceCatalog, namespace, pattern) case ShowNamespacesStatement(None, pattern) => @@ -186,7 +186,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) if (isNamespaceSet) { SetCatalogAndNamespace(catalogManager, None, Some(nameParts)) } else { - val CurrentCatalogAndNamespace(catalog, namespace) = nameParts + val CatalogAndNamespace(catalog, namespace) = nameParts SetCatalogAndNamespace(catalogManager, Some(catalog.name()), namespace) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 1b5197f31842..1cadff5d7ca7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -84,32 +84,14 @@ private[sql] trait LookupCatalog extends Logging { } } - type DefaultCatalogAndNamespace = (Option[CatalogPlugin], Seq[String]) - - /** - * Extract catalog and namespace from a multi-part identifier with the default catalog if needed. - * Catalog name takes precedence over namespaces. - */ - object DefaultCatalogAndNamespace { - def unapply(parts: Seq[String]): Some[DefaultCatalogAndNamespace] = parts match { - case Seq(catalogName, tail @ _*) => - try { - Some((Some(catalogManager.catalog(catalogName)), tail)) - } catch { - case _: CatalogNotFoundException => - Some((defaultCatalog, parts)) - } - } - } - - type CurrentCatalogAndNamespace = (CatalogPlugin, Option[Seq[String]]) + type CatalogAndNamespace = (CatalogPlugin, Option[Seq[String]]) /** * Extract catalog and namespace from a multi-part identifier with the current catalog if needed. * Catalog name takes precedence over namespaces. */ - object CurrentCatalogAndNamespace { - def unapply(parts: Seq[String]): Some[CurrentCatalogAndNamespace] = parts match { + object CatalogAndNamespace { + def unapply(parts: Seq[String]): Some[CatalogAndNamespace] = parts match { case Seq(catalogName, tail @ _*) => try { Some( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 57877f51a0a5..1afe3976b2a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2694,7 +2694,7 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession { sparkContext.addSparkListener(listener) try { // Execute the command. - sql("EXPLAIN show databases").head() + sql("show databases").head() // Make sure we have seen all events triggered by DataFrame.show() sparkContext.listenerBus.waitUntilEmpty() From d8b53624ab974e80e522c30ab2731a89cab1fa8f Mon Sep 17 00:00:00 2001 From: Terry Kim Date: Wed, 9 Oct 2019 21:37:48 -0700 Subject: [PATCH 5/5] Address PR comments --- .../apache/spark/sql/connector/catalog/LookupCatalog.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala index 1cadff5d7ca7..02585fd5c463 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala @@ -84,14 +84,12 @@ private[sql] trait LookupCatalog extends Logging { } } - type CatalogAndNamespace = (CatalogPlugin, Option[Seq[String]]) - /** * Extract catalog and namespace from a multi-part identifier with the current catalog if needed. * Catalog name takes precedence over namespaces. */ object CatalogAndNamespace { - def unapply(parts: Seq[String]): Some[CatalogAndNamespace] = parts match { + def unapply(parts: Seq[String]): Some[(CatalogPlugin, Option[Seq[String]])] = parts match { case Seq(catalogName, tail @ _*) => try { Some(