From 65daf5d91941310f3e48074baf8bfd501143bfdc Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 10:40:33 +0300 Subject: [PATCH 01/40] Add ShowPartitionsExec --- .../datasources/v2/ShowPartitionsExec.scala | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala new file mode 100644 index 000000000000..5e4850999831 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -0,0 +1,38 @@ +/* + * 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.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.execution.LeafExecNode + +/** + * Physical plan node for showing partitions. + */ +case class ShowPartitionsExec( + output: Seq[Attribute], + catalog: TableCatalog, + namespace: Seq[String], + pattern: Option[String]) extends V2CommandExec with LeafExecNode { + override protected def run(): Seq[InternalRow] = { + // scalastyle:off + throw new NotImplementedError("") + // scalastyle:on + } +} From ef84a7377e84740a09b8dddafb7fcfad2053b916 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 11:45:52 +0300 Subject: [PATCH 02/40] Add logical node ShowPartitions --- .../sql/catalyst/plans/logical/v2Commands.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 5bda2b5b8db0..e4d0c407fc88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -670,3 +670,15 @@ case class LoadData( case class ShowCreateTable(child: LogicalPlan, asSerde: Boolean = false) extends Command { override def children: Seq[LogicalPlan] = child :: Nil } + +/** + * The logical plan of the SHOW PARTITIONS command. + */ +case class ShowPartitions( + namespace: LogicalPlan, + spec: Option[TablePartitionSpec]) extends Command { + override def children: Seq[LogicalPlan] = Seq(namespace) + + override val output: Seq[Attribute] = Seq( + AttributeReference("partition", StringType, nullable = false)()) +} From 6934fa13cf1101487f5afd698b6f70c29f604e64 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 12:50:20 +0300 Subject: [PATCH 03/40] ShowPartitionsStatement -> ShowPartitions --- .../spark/sql/catalyst/parser/AstBuilder.scala | 6 ++++-- .../sql/catalyst/plans/logical/statements.scala | 7 ------- .../analysis/ResolveSessionCatalog.scala | 5 ++--- .../command/ShowPartitionsParserSuite.scala | 16 ++++++++-------- 4 files changed, 14 insertions(+), 20 deletions(-) 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 c3855fe088db..98d22860b420 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 @@ -3366,9 +3366,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg * }}} */ override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) { - val table = visitMultipartIdentifier(ctx.multipartIdentifier) + val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) - ShowPartitionsStatement(table, partitionKeys) + ShowPartitions( + UnresolvedNamespace(multiPart.getOrElse(Seq.empty[String])), + partitionKeys) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index 39bc5a5604b2..65f72142e517 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -352,13 +352,6 @@ case class TruncateTableStatement( tableName: Seq[String], partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement -/** - * A SHOW PARTITIONS statement, as parsed from SQL - */ -case class ShowPartitionsStatement( - tableName: Seq[String], - partitionSpec: Option[TablePartitionSpec]) extends ParsedStatement - /** * A SHOW COLUMNS statement, as parsed from SQL */ 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 bd9120a1fbe7..3e29515e3baf 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 @@ -462,10 +462,9 @@ class ResolveSessionCatalog( v1TableName.asTableIdentifier, partitionSpec) - case ShowPartitionsStatement(tbl, partitionSpec) => - val v1TableName = parseV1Table(tbl, "SHOW PARTITIONS") + case ShowPartitions(SessionCatalogAndNamespace(_, ns), partitionSpec) => ShowPartitionsCommand( - v1TableName.asTableIdentifier, + parseV1Table(ns, "SHOW PARTITIONS").asTableIdentifier, partitionSpec) case ShowColumnsStatement(tbl, ns) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala index bc75528b9644..2ec308c48eb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala @@ -17,25 +17,25 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.catalyst.analysis.AnalysisTest +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.ShowPartitionsStatement +import org.apache.spark.sql.catalyst.plans.logical.ShowPartitions import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.test.SharedSparkSession class ShowPartitionsParserSuite extends AnalysisTest with SharedSparkSession { test("SHOW PARTITIONS") { Seq( - "SHOW PARTITIONS t1" -> ShowPartitionsStatement(Seq("t1"), None), - "SHOW PARTITIONS db1.t1" -> ShowPartitionsStatement(Seq("db1", "t1"), None), + "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedNamespace(Seq("t1")), None), + "SHOW PARTITIONS db1.t1" -> ShowPartitions(UnresolvedNamespace(Seq("db1", "t1")), None), "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" -> - ShowPartitionsStatement( - Seq("t1"), + ShowPartitions( + UnresolvedNamespace(Seq("t1")), Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue"))), - "SHOW PARTITIONS a.b.c" -> ShowPartitionsStatement(Seq("a", "b", "c"), None), + "SHOW PARTITIONS a.b.c" -> ShowPartitions(UnresolvedNamespace(Seq("a", "b", "c")), None), "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" -> - ShowPartitionsStatement(Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10"))) + ShowPartitions(UnresolvedNamespace(Seq("a", "b", "c")), Some(Map("ds" -> "2017-06-10"))) ).foreach { case (sql, expected) => val parsed = parsePlan(sql) comparePlans(parsed, expected) From c62b30bd10de8dd98531ac6784a1950371078f6b Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 13:16:26 +0300 Subject: [PATCH 04/40] Resolve ShowPartitions -> ShowPartitionsExec --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 ++ .../sql/execution/datasources/v2/DataSourceV2Strategy.scala | 3 +++ .../sql/execution/datasources/v2/ShowPartitionsExec.scala | 5 +++-- .../spark/sql/execution/command/v2/ShowPartitionsSuite.scala | 4 ++-- 4 files changed, 10 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 14b50f481f38..ab11c8964a85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -831,6 +831,8 @@ class Analyzer(override val catalogManager: CatalogManager) case class ResolveNamespace(catalogManager: CatalogManager) extends Rule[LogicalPlan] with LookupCatalog { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case s @ ShowPartitions(UnresolvedNamespace(Seq()), _) => + s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) case s @ ShowTables(UnresolvedNamespace(Seq()), _) => s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) case s @ ShowViews(UnresolvedNamespace(Seq()), _) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 21abfc2816ee..1d528fea6e21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -299,6 +299,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case ShowCreateTable(_: ResolvedTable, _) => throw new AnalysisException("SHOW CREATE TABLE is not supported for v2 tables.") + case r @ ShowPartitions(ResolvedNamespace(catalog, ns), pattern) => + ShowPartitionsExec(r.output, catalog.asTableCatalog, ns, pattern) :: Nil + case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 5e4850999831..0015d214c802 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.execution.LeafExecNode @@ -29,10 +30,10 @@ case class ShowPartitionsExec( output: Seq[Attribute], catalog: TableCatalog, namespace: Seq[String], - pattern: Option[String]) extends V2CommandExec with LeafExecNode { + spec: Option[TablePartitionSpec]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { // scalastyle:off - throw new NotImplementedError("") + throw new NotImplementedError("SHOW PARTITIONS is not implemented") // scalastyle:on } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 8a63cd49e89e..5c4afb158479 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -35,10 +35,10 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa // TODO(SPARK-33452): Create a V2 SHOW PARTITIONS execution node test("not supported SHOW PARTITIONS") { def testV1Command(sqlCommand: String, sqlParams: String): Unit = { - val e = intercept[AnalysisException] { + val e = intercept[NotImplementedError] { sql(s"$sqlCommand $sqlParams") } - assert(e.message.contains(s"$sqlCommand is only supported with v1 tables")) + assert(e.getMessage.contains("SHOW PARTITIONS is not implemented")) } val t = s"$catalog.ns1.ns2.tbl" withTable(t) { From d1190c7e456af819da477bb7e609cde33802f2d7 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 18:35:43 +0300 Subject: [PATCH 05/40] Remove `verify(s"SHOW PARTITIONS $t")` --- .../org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala | 1 - 1 file changed, 1 deletion(-) 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 5f7be7c4c565..cf6ab8cfd5e4 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 @@ -2256,7 +2256,6 @@ class DataSourceV2SQLSuite verify(s"CACHE TABLE $t") verify(s"UNCACHE TABLE $t") verify(s"TRUNCATE TABLE $t") - verify(s"SHOW PARTITIONS $t") verify(s"SHOW COLUMNS FROM $t") } } From e086c32dd2a0c6251d2af721b69bd22b7fbfa28e Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 20:40:30 +0300 Subject: [PATCH 06/40] Pass `UnresolvedTableOrView` to `ShowPartitions` --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 -- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 2 +- .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 6 ++---- .../execution/datasources/v2/DataSourceV2Strategy.scala | 9 ++++++--- .../execution/datasources/v2/ShowPartitionsExec.scala | 5 +++-- .../sql/execution/command/v1/ShowPartitionsSuite.scala | 2 ++ .../sql/execution/command/v2/ShowPartitionsSuite.scala | 1 - 7 files changed, 14 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index ab11c8964a85..14b50f481f38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -831,8 +831,6 @@ class Analyzer(override val catalogManager: CatalogManager) case class ResolveNamespace(catalogManager: CatalogManager) extends Rule[LogicalPlan] with LookupCatalog { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case s @ ShowPartitions(UnresolvedNamespace(Seq()), _) => - s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) case s @ ShowTables(UnresolvedNamespace(Seq()), _) => s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) case s @ ShowViews(UnresolvedNamespace(Seq()), _) => 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 98d22860b420..10e5d26e9a06 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 @@ -3369,7 +3369,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) ShowPartitions( - UnresolvedNamespace(multiPart.getOrElse(Seq.empty[String])), + UnresolvedTableOrView(multiPart.getOrElse(Seq.empty[String])), partitionKeys) } 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 3e29515e3baf..d6df18d22e43 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 @@ -462,10 +462,8 @@ class ResolveSessionCatalog( v1TableName.asTableIdentifier, partitionSpec) - case ShowPartitions(SessionCatalogAndNamespace(_, ns), partitionSpec) => - ShowPartitionsCommand( - parseV1Table(ns, "SHOW PARTITIONS").asTableIdentifier, - partitionSpec) + case ShowPartitions(ResolvedV1TableOrViewIdentifier(ident), partitionSpec) => + ShowPartitionsCommand(ident.asTableIdentifier, partitionSpec) case ShowColumnsStatement(tbl, ns) => if (ns.isDefined && ns.get.length > 1) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 1d528fea6e21..296242753b6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy} -import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable} +import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable, ResolvedView} import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ @@ -299,8 +299,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case ShowCreateTable(_: ResolvedTable, _) => throw new AnalysisException("SHOW CREATE TABLE is not supported for v2 tables.") - case r @ ShowPartitions(ResolvedNamespace(catalog, ns), pattern) => - ShowPartitionsExec(r.output, catalog.asTableCatalog, ns, pattern) :: Nil + case r @ ShowPartitions(t: ResolvedTable, pattern) => + ShowPartitionsExec(r.output, t.catalog, t.identifier, t.table, pattern) :: Nil + + case ShowPartitions(v: ResolvedView, _) => + throw new AnalysisException(s"SHOW PARTITIONS is not allowed on a view: ${v.identifier}") case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 0015d214c802..0ef056a6aea9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} import org.apache.spark.sql.execution.LeafExecNode /** @@ -29,7 +29,8 @@ import org.apache.spark.sql.execution.LeafExecNode case class ShowPartitionsExec( output: Seq[Attribute], catalog: TableCatalog, - namespace: Seq[String], + identifier: Identifier, + table: Table, spec: Option[TablePartitionSpec]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { // scalastyle:off diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index bcc71e9b7241..84dc3cdd5a2e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -124,6 +124,8 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { } } + // The test fails for V2 Table Catalogs with the exception: + // org.apache.spark.sql.AnalysisException: CREATE VIEW is only supported with v1 tables. test("show partitions of a view") { val table = "dateTable" withTable(table) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 5c4afb158479..d4f7fa3a5ca8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.SparkConf -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.connector.InMemoryTableCatalog import org.apache.spark.sql.execution.command import org.apache.spark.sql.test.SharedSparkSession From 7e0638103de4cd0e9382eadb3fc3a1391f0d565f Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 20:58:40 +0300 Subject: [PATCH 07/40] Throw an exception for non-partitioned table --- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../datasources/v2/ShowPartitionsExec.scala | 15 +++++++++++++-- .../command/ShowPartitionsSuiteBase.scala | 16 +++++++++++++++- .../command/v1/ShowPartitionsSuite.scala | 11 ----------- 4 files changed, 29 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 296242753b6d..7975b25d536e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -300,7 +300,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat throw new AnalysisException("SHOW CREATE TABLE is not supported for v2 tables.") case r @ ShowPartitions(t: ResolvedTable, pattern) => - ShowPartitionsExec(r.output, t.catalog, t.identifier, t.table, pattern) :: Nil + ShowPartitionsExec(r.output, t.catalog, t.table, pattern) :: Nil case ShowPartitions(v: ResolvedView, _) => throw new AnalysisException(s"SHOW PARTITIONS is not allowed on a view: ${v.identifier}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 0ef056a6aea9..309613da65cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.execution.datasources.v2 +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{Table, TableCatalog} import org.apache.spark.sql.execution.LeafExecNode /** @@ -29,10 +30,20 @@ import org.apache.spark.sql.execution.LeafExecNode case class ShowPartitionsExec( output: Seq[Attribute], catalog: TableCatalog, - identifier: Identifier, table: Table, spec: Option[TablePartitionSpec]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { + /** + * Validate and throws an [[AnalysisException]] exception under the following conditions: + * 1. If the table is not partitioned. + * 2. If it is a datasource table. + * 3. If it is a view. + */ + if (table.partitioning.isEmpty) { + throw new AnalysisException( + s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${table.name()}") + } + // scalastyle:off throw new NotImplementedError("SHOW PARTITIONS is not implemented") // scalastyle:on diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 413e170326ee..9e6a16894b80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import org.scalactic.source.Position import org.scalatest.Tag -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.test.SQLTestUtils trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { @@ -33,4 +33,18 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { (implicit pos: Position): Unit = { super.test(s"SHOW PARTITIONS $version: " + testName, testTags: _*)(testFun) } + + test("show partitions of non-partitioned table") { + withNamespace(s"$catalog.ns") { + sql(s"CREATE NAMESPACE $catalog.ns") + val table = s"$catalog.ns.not_partitioned_table" + withTable(table) { + sql(s"CREATE TABLE $table (col1 int) $defaultUsing") + val errMsg = intercept[AnalysisException] { + sql(s"SHOW PARTITIONS $table") + }.getMessage + assert(errMsg.contains("not allowed on a table that is not partitioned")) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 84dc3cdd5a2e..e57b481aa4c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -113,17 +113,6 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { } } - test("show partitions of non-partitioned table") { - val table = "not_partitioned_table" - withTable(table) { - sql(s"CREATE TABLE $table (col1 int) $defaultUsing") - val errMsg = intercept[AnalysisException] { - sql(s"SHOW PARTITIONS $table") - }.getMessage - assert(errMsg.contains("not allowed on a table that is not partitioned")) - } - } - // The test fails for V2 Table Catalogs with the exception: // org.apache.spark.sql.AnalysisException: CREATE VIEW is only supported with v1 tables. test("show partitions of a view") { From 9cbbbe193c3c0bbd46dd90c2d876724aa9d16569 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 21:01:20 +0300 Subject: [PATCH 08/40] Fix ShowPartitionsParserSuite --- .../command/ShowPartitionsParserSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala index 2ec308c48eb7..beb08940590b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedTableOrView} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.ShowPartitions @@ -27,15 +27,15 @@ import org.apache.spark.sql.test.SharedSparkSession class ShowPartitionsParserSuite extends AnalysisTest with SharedSparkSession { test("SHOW PARTITIONS") { Seq( - "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedNamespace(Seq("t1")), None), - "SHOW PARTITIONS db1.t1" -> ShowPartitions(UnresolvedNamespace(Seq("db1", "t1")), None), + "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedTableOrView(Seq("t1")), None), + "SHOW PARTITIONS db1.t1" -> ShowPartitions(UnresolvedTableOrView(Seq("db1", "t1")), None), "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" -> ShowPartitions( - UnresolvedNamespace(Seq("t1")), + UnresolvedTableOrView(Seq("t1")), Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue"))), - "SHOW PARTITIONS a.b.c" -> ShowPartitions(UnresolvedNamespace(Seq("a", "b", "c")), None), + "SHOW PARTITIONS a.b.c" -> ShowPartitions(UnresolvedTableOrView(Seq("a", "b", "c")), None), "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" -> - ShowPartitions(UnresolvedNamespace(Seq("a", "b", "c")), Some(Map("ds" -> "2017-06-10"))) + ShowPartitions(UnresolvedTableOrView(Seq("a", "b", "c")), Some(Map("ds" -> "2017-06-10"))) ).foreach { case (sql, expected) => val parsed = parsePlan(sql) comparePlans(parsed, expected) From 04c9317382cf489d2003712adaa0684f9c81305b Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 22:55:35 +0300 Subject: [PATCH 09/40] Ignore test in PartitionedTablePerfStatsSuite --- .../spark/sql/hive/PartitionedTablePerfStatsSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala index 3af163af0968..62db0f86110d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala @@ -283,7 +283,8 @@ class PartitionedTablePerfStatsSuite } } - test("hive table: num hive client calls does not scale with partition count") { + // FIXME: SPARK-33452 + ignore("hive table: num hive client calls does not scale with partition count") { withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { withTable("test") { withTempDir { dir => @@ -306,7 +307,8 @@ class PartitionedTablePerfStatsSuite } } - test("datasource table: num hive client calls does not scale with partition count") { + // FIXME: SPARK-33452 + ignore("datasource table: num hive client calls does not scale with partition count") { withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { withTable("test") { withTempDir { dir => From ed915c9c29c3aa595cb95bb8ca331edca949ea95 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 23:28:19 +0300 Subject: [PATCH 10/40] Check partition spec --- .../datasources/v2/ShowPartitionsExec.scala | 18 +++++++++++++ .../command/ShowPartitionsSuiteBase.scala | 25 +++++++++++++++++++ .../command/v1/ShowPartitionsSuite.scala | 22 ---------------- 3 files changed, 43 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 309613da65cc..cdd468b54c5b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -44,6 +44,24 @@ case class ShowPartitionsExec( s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${table.name()}") } + /** + * Validate the partitioning spec by making sure all the referenced columns are + * defined as partitioning columns in table definition. An AnalysisException exception is + * thrown if the partitioning spec is invalid. + */ + if (spec.isDefined) { + val partitionColumnNames = + table.partitioning().flatMap(_.references.flatMap(_.fieldNames)).toSet + val specKeys = spec.get.keySet + if (!specKeys.subsetOf(partitionColumnNames)) { + val badColumns = specKeys + .filterNot(partitionColumnNames.contains) + .mkString("[", ", ", "]") + throw new AnalysisException( + s"Non-partitioning column(s) $badColumns are specified for SHOW PARTITIONS") + } + } + // scalastyle:off throw new NotImplementedError("SHOW PARTITIONS is not implemented") // scalastyle:on diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 9e6a16894b80..4323bf95b93c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -34,6 +34,17 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { super.test(s"SHOW PARTITIONS $version: " + testName, testTags: _*)(testFun) } + protected def createDateTable(table: String): Unit = { + sql(s""" + |CREATE TABLE $table (price int, qty int, year int, month int) + |$defaultUsing + |partitioned by (year, month)""".stripMargin) + sql(s"INSERT INTO $table PARTITION(year = 2015, month = 1) SELECT 1, 1") + sql(s"INSERT INTO $table PARTITION(year = 2015, month = 2) SELECT 2, 2") + sql(s"INSERT INTO $table PARTITION(year = 2016, month = 2) SELECT 3, 3") + sql(s"INSERT INTO $table PARTITION(year = 2016, month = 3) SELECT 3, 3") + } + test("show partitions of non-partitioned table") { withNamespace(s"$catalog.ns") { sql(s"CREATE NAMESPACE $catalog.ns") @@ -47,4 +58,18 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { } } } + + test("non-partitioning columns") { + withNamespace(s"$catalog.ns") { + sql(s"CREATE NAMESPACE $catalog.ns") + val table = s"$catalog.ns.dateTable" + withTable(table) { + createDateTable(table) + val errMsg = intercept[AnalysisException] { + sql(s"SHOW PARTITIONS $table PARTITION(abcd=2015, xyz=1)") + }.getMessage + assert(errMsg.contains("Non-partitioning column(s) [abcd, xyz] are specified")) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index e57b481aa4c8..d64d9f26488a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -29,17 +29,6 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { override def defaultNamespace: Seq[String] = Seq("default") override def defaultUsing: String = "USING parquet" - private def createDateTable(table: String): Unit = { - sql(s""" - |CREATE TABLE $table (price int, qty int, year int, month int) - |$defaultUsing - |partitioned by (year, month)""".stripMargin) - sql(s"INSERT INTO $table PARTITION(year = 2015, month = 1) SELECT 1, 1") - sql(s"INSERT INTO $table PARTITION(year = 2015, month = 2) SELECT 2, 2") - sql(s"INSERT INTO $table PARTITION(year = 2016, month = 2) SELECT 3, 3") - sql(s"INSERT INTO $table PARTITION(year = 2016, month = 3) SELECT 3, 3") - } - test("show everything") { val table = "dateTable" withTable(table) { @@ -102,17 +91,6 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { } } - test("non-partitioning columns") { - val table = "dateTable" - withTable(table) { - createDateTable(table) - val errMsg = intercept[AnalysisException] { - sql(s"SHOW PARTITIONS $table PARTITION(abcd=2015, xyz=1)") - }.getMessage - assert(errMsg.contains("Non-partitioning column(s) [abcd, xyz] are specified")) - } - } - // The test fails for V2 Table Catalogs with the exception: // org.apache.spark.sql.AnalysisException: CREATE VIEW is only supported with v1 tables. test("show partitions of a view") { From ade84ba4dbe6ee4e6589d93f3d3b1d8c3af72298 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 17 Nov 2020 23:31:55 +0300 Subject: [PATCH 11/40] Remove a comment --- .../sql/execution/datasources/v2/ShowPartitionsExec.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index cdd468b54c5b..8d1ac434dc37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -33,12 +33,6 @@ case class ShowPartitionsExec( table: Table, spec: Option[TablePartitionSpec]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { - /** - * Validate and throws an [[AnalysisException]] exception under the following conditions: - * 1. If the table is not partitioned. - * 2. If it is a datasource table. - * 3. If it is a view. - */ if (table.partitioning.isEmpty) { throw new AnalysisException( s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${table.name()}") From 3a457f9a8622eb38305b93fc300cb6c328c53880 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 18 Nov 2020 10:37:51 +0300 Subject: [PATCH 12/40] pass Table to ShowPartitionsExec --- .../datasources/v2/DataSourceV2Strategy.scala | 5 ++--- .../datasources/v2/ShowPartitionsExec.scala | 12 +++++------- .../execution/command/ShowPartitionsSuiteBase.scala | 6 +++--- .../execution/command/v2/ShowPartitionsSuite.scala | 4 ++-- 4 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 7975b25d536e..82fd49042969 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -299,11 +299,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case ShowCreateTable(_: ResolvedTable, _) => throw new AnalysisException("SHOW CREATE TABLE is not supported for v2 tables.") - case r @ ShowPartitions(t: ResolvedTable, pattern) => - ShowPartitionsExec(r.output, t.catalog, t.table, pattern) :: Nil - case ShowPartitions(v: ResolvedView, _) => throw new AnalysisException(s"SHOW PARTITIONS is not allowed on a view: ${v.identifier}") + case r @ ShowPartitions(t: ResolvedTable, pattern) => + ShowPartitionsExec(r.output, t.catalog, t.table, pattern) :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 8d1ac434dc37..19f8b7f32468 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -45,14 +45,12 @@ case class ShowPartitionsExec( */ if (spec.isDefined) { val partitionColumnNames = - table.partitioning().flatMap(_.references.flatMap(_.fieldNames)).toSet - val specKeys = spec.get.keySet - if (!specKeys.subsetOf(partitionColumnNames)) { - val badColumns = specKeys - .filterNot(partitionColumnNames.contains) - .mkString("[", ", ", "]") + table.partitioning().flatMap(_.references.flatMap(_.fieldNames)) + val badColumns = spec.get.keySet.filterNot(partitionColumnNames.contains) + if (badColumns.nonEmpty) { + val badCols = badColumns.mkString("[", ", ", "]") throw new AnalysisException( - s"Non-partitioning column(s) $badColumns are specified for SHOW PARTITIONS") + s"Non-partitioning column(s) $badCols are specified for SHOW PARTITIONS") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 4323bf95b93c..004c681bf65d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -36,9 +36,9 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { protected def createDateTable(table: String): Unit = { sql(s""" - |CREATE TABLE $table (price int, qty int, year int, month int) - |$defaultUsing - |partitioned by (year, month)""".stripMargin) + |CREATE TABLE $table (price int, qty int, year int, month int) + |$defaultUsing + |partitioned by (year, month)""".stripMargin) sql(s"INSERT INTO $table PARTITION(year = 2015, month = 1) SELECT 1, 1") sql(s"INSERT INTO $table PARTITION(year = 2015, month = 2) SELECT 2, 2") sql(s"INSERT INTO $table PARTITION(year = 2016, month = 2) SELECT 3, 3") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index d4f7fa3a5ca8..325a7af90ae0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.SparkConf -import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog import org.apache.spark.sql.execution.command import org.apache.spark.sql.test.SharedSparkSession @@ -29,7 +29,7 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa override def defaultUsing: String = "USING _" override def sparkConf: SparkConf = super.sparkConf - .set(s"spark.sql.catalog.$catalog", classOf[InMemoryTableCatalog].getName) + .set(s"spark.sql.catalog.$catalog", classOf[InMemoryPartitionTableCatalog].getName) // TODO(SPARK-33452): Create a V2 SHOW PARTITIONS execution node test("not supported SHOW PARTITIONS") { From 8700ed5b77c37e172601572baf2a5ec7ddd43b14 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 18 Nov 2020 14:37:45 +0300 Subject: [PATCH 13/40] Change type of partition spec in ShowPartitions --- .../analysis/ResolvePartitionSpec.scala | 5 ++- .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../catalyst/plans/logical/v2Commands.scala | 9 +++-- .../analysis/ResolveSessionCatalog.scala | 8 +++- .../datasources/v2/DataSourceV2Strategy.scala | 12 ++++-- .../datasources/v2/ShowPartitionsExec.scala | 24 +++--------- .../command/ShowPartitionsParserSuite.scala | 8 ++-- .../command/ShowPartitionsSuiteBase.scala | 25 ------------ .../command/v1/ShowPartitionsSuite.scala | 38 ++++++++++--------- .../command/v2/ShowPartitionsSuite.scala | 2 +- 10 files changed, 59 insertions(+), 76 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index 5e19a3296899..bde60b10d407 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.plans.logical.{AlterTableAddPartition, AlterTableDropPartition, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTableAddPartition, AlterTableDropPartition, LogicalPlan, ShowPartitions} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement import org.apache.spark.sql.types._ @@ -38,6 +38,9 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { case r @ AlterTableDropPartition( ResolvedTable(_, _, table: SupportsPartitionManagement), partSpecs, _, _, _) => r.copy(parts = resolvePartitionSpecs(partSpecs, table.partitionSchema())) + + case r @ ShowPartitions(ResolvedTable(_, _, table: SupportsPartitionManagement), partSpecs) => + r.copy(pattern = resolvePartitionSpecs(partSpecs.toSeq, table.partitionSchema()).headOption) } private def resolvePartitionSpecs( 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 10e5d26e9a06..d750937d4684 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 @@ -3367,7 +3367,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) { val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) - val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) + val partitionKeys = Option(ctx.partitionSpec).map { specCtx => + UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) + } ShowPartitions( UnresolvedTableOrView(multiPart.getOrElse(Seq.empty[String])), partitionKeys) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index e4d0c407fc88..1920ab69960b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -675,9 +675,12 @@ case class ShowCreateTable(child: LogicalPlan, asSerde: Boolean = false) extends * The logical plan of the SHOW PARTITIONS command. */ case class ShowPartitions( - namespace: LogicalPlan, - spec: Option[TablePartitionSpec]) extends Command { - override def children: Seq[LogicalPlan] = Seq(namespace) + child: LogicalPlan, + pattern: Option[PartitionSpec]) extends Command { + override def children: Seq[LogicalPlan] = child :: Nil + + override lazy val resolved: Boolean = + childrenResolved && pattern.forall(_.isInstanceOf[ResolvedPartitionSpec]) override val output: Seq[Attribute] = Seq( AttributeReference("partition", StringType, nullable = false)()) 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 d6df18d22e43..8471fa452297 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 @@ -462,8 +462,12 @@ class ResolveSessionCatalog( v1TableName.asTableIdentifier, partitionSpec) - case ShowPartitions(ResolvedV1TableOrViewIdentifier(ident), partitionSpec) => - ShowPartitionsCommand(ident.asTableIdentifier, partitionSpec) + case ShowPartitions( + ResolvedV1TableOrViewIdentifier(ident), + pattern @ (None | Some(UnresolvedPartitionSpec(_, _)))) => + ShowPartitionsCommand( + ident.asTableIdentifier, + pattern.map(_.asInstanceOf[UnresolvedPartitionSpec].spec)) case ShowColumnsStatement(tbl, ns) => if (ns.isDefined && ns.get.length > 1) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 82fd49042969..0c3b9fc510bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy} -import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedTable, ResolvedView} +import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable, ResolvedView} import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ @@ -301,8 +301,14 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case ShowPartitions(v: ResolvedView, _) => throw new AnalysisException(s"SHOW PARTITIONS is not allowed on a view: ${v.identifier}") - case r @ ShowPartitions(t: ResolvedTable, pattern) => - ShowPartitionsExec(r.output, t.catalog, t.table, pattern) :: Nil + case r @ ShowPartitions( + ResolvedTable(catalog, _, table: SupportsPartitionManagement), + pattern @ (None | Some(ResolvedPartitionSpec(_, _)))) => + ShowPartitionsExec( + r.output, + catalog, + table, + pattern.map(_.asInstanceOf[ResolvedPartitionSpec])) :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 19f8b7f32468..9665041f8088 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.{Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{SupportsPartitionManagement, TableCatalog} import org.apache.spark.sql.execution.LeafExecNode /** @@ -30,29 +30,15 @@ import org.apache.spark.sql.execution.LeafExecNode case class ShowPartitionsExec( output: Seq[Attribute], catalog: TableCatalog, - table: Table, - spec: Option[TablePartitionSpec]) extends V2CommandExec with LeafExecNode { + table: SupportsPartitionManagement, + spec: Option[ResolvedPartitionSpec]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { if (table.partitioning.isEmpty) { throw new AnalysisException( s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${table.name()}") } - /** - * Validate the partitioning spec by making sure all the referenced columns are - * defined as partitioning columns in table definition. An AnalysisException exception is - * thrown if the partitioning spec is invalid. - */ - if (spec.isDefined) { - val partitionColumnNames = - table.partitioning().flatMap(_.references.flatMap(_.fieldNames)) - val badColumns = spec.get.keySet.filterNot(partitionColumnNames.contains) - if (badColumns.nonEmpty) { - val badCols = badColumns.mkString("[", ", ", "]") - throw new AnalysisException( - s"Non-partitioning column(s) $badCols are specified for SHOW PARTITIONS") - } - } + // table.listPartitionIdentifiers(spec.get.spec) // scalastyle:off throw new NotImplementedError("SHOW PARTITIONS is not implemented") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala index beb08940590b..a382f6c3a716 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedTableOrView} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedPartitionSpec, UnresolvedTableOrView} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.ShowPartitions @@ -32,10 +32,12 @@ class ShowPartitionsParserSuite extends AnalysisTest with SharedSparkSession { "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" -> ShowPartitions( UnresolvedTableOrView(Seq("t1")), - Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue"))), + Some(UnresolvedPartitionSpec(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue")))), "SHOW PARTITIONS a.b.c" -> ShowPartitions(UnresolvedTableOrView(Seq("a", "b", "c")), None), "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" -> - ShowPartitions(UnresolvedTableOrView(Seq("a", "b", "c")), Some(Map("ds" -> "2017-06-10"))) + ShowPartitions( + UnresolvedTableOrView(Seq("a", "b", "c")), + Some(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")))) ).foreach { case (sql, expected) => val parsed = parsePlan(sql) comparePlans(parsed, expected) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 004c681bf65d..9e6a16894b80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -34,17 +34,6 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { super.test(s"SHOW PARTITIONS $version: " + testName, testTags: _*)(testFun) } - protected def createDateTable(table: String): Unit = { - sql(s""" - |CREATE TABLE $table (price int, qty int, year int, month int) - |$defaultUsing - |partitioned by (year, month)""".stripMargin) - sql(s"INSERT INTO $table PARTITION(year = 2015, month = 1) SELECT 1, 1") - sql(s"INSERT INTO $table PARTITION(year = 2015, month = 2) SELECT 2, 2") - sql(s"INSERT INTO $table PARTITION(year = 2016, month = 2) SELECT 3, 3") - sql(s"INSERT INTO $table PARTITION(year = 2016, month = 3) SELECT 3, 3") - } - test("show partitions of non-partitioned table") { withNamespace(s"$catalog.ns") { sql(s"CREATE NAMESPACE $catalog.ns") @@ -58,18 +47,4 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { } } } - - test("non-partitioning columns") { - withNamespace(s"$catalog.ns") { - sql(s"CREATE NAMESPACE $catalog.ns") - val table = s"$catalog.ns.dateTable" - withTable(table) { - createDateTable(table) - val errMsg = intercept[AnalysisException] { - sql(s"SHOW PARTITIONS $table PARTITION(abcd=2015, xyz=1)") - }.getMessage - assert(errMsg.contains("Non-partitioning column(s) [abcd, xyz] are specified")) - } - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index d64d9f26488a..18a87972423f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -29,24 +29,15 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { override def defaultNamespace: Seq[String] = Seq("default") override def defaultUsing: String = "USING parquet" - test("show everything") { - val table = "dateTable" - withTable(table) { - createDateTable(table) - checkAnswer( - sql(s"show partitions $table"), - Row("year=2015/month=1") :: - Row("year=2015/month=2") :: - Row("year=2016/month=2") :: - Row("year=2016/month=3") :: Nil) - - checkAnswer( - sql(s"show partitions default.$table"), - Row("year=2015/month=1") :: - Row("year=2015/month=2") :: - Row("year=2016/month=2") :: - Row("year=2016/month=3") :: Nil) - } + private def createDateTable(table: String): Unit = { + sql(s""" + |CREATE TABLE $table (price int, qty int, year int, month int) + |$defaultUsing + |partitioned by (year, month)""".stripMargin) + sql(s"INSERT INTO $table PARTITION(year = 2015, month = 1) SELECT 1, 1") + sql(s"INSERT INTO $table PARTITION(year = 2015, month = 2) SELECT 2, 2") + sql(s"INSERT INTO $table PARTITION(year = 2016, month = 2) SELECT 3, 3") + sql(s"INSERT INTO $table PARTITION(year = 2016, month = 3) SELECT 3, 3") } test("filter by partitions") { @@ -91,6 +82,17 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { } } + test("non-partitioning columns") { + val table = "dateTable" + withTable(table) { + createDateTable(table) + val errMsg = intercept[AnalysisException] { + sql(s"SHOW PARTITIONS $table PARTITION(abcd=2015, xyz=1)") + }.getMessage + assert(errMsg.contains("Non-partitioning column(s) [abcd, xyz] are specified")) + } + } + // The test fails for V2 Table Catalogs with the exception: // org.apache.spark.sql.AnalysisException: CREATE VIEW is only supported with v1 tables. test("show partitions of a view") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 325a7af90ae0..35a8555a7e9c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -37,7 +37,7 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa val e = intercept[NotImplementedError] { sql(s"$sqlCommand $sqlParams") } - assert(e.getMessage.contains("SHOW PARTITIONS is not implemented")) + assert(e.getMessage.contains(s"SHOW PARTITIONS is not implemented")) } val t = s"$catalog.ns1.ns2.tbl" withTable(t) { From dfbad2525b4c5682c25d400ab9f06fed98ce27f6 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 18 Nov 2020 17:32:56 +0300 Subject: [PATCH 14/40] Revert a test in v1/ShowPartitionsSuite --- .../command/v1/ShowPartitionsSuite.scala | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 18a87972423f..e57b481aa4c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -40,6 +40,26 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { sql(s"INSERT INTO $table PARTITION(year = 2016, month = 3) SELECT 3, 3") } + test("show everything") { + val table = "dateTable" + withTable(table) { + createDateTable(table) + checkAnswer( + sql(s"show partitions $table"), + Row("year=2015/month=1") :: + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: + Row("year=2016/month=3") :: Nil) + + checkAnswer( + sql(s"show partitions default.$table"), + Row("year=2015/month=1") :: + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: + Row("year=2016/month=3") :: Nil) + } + } + test("filter by partitions") { val table = "dateTable" withTable(table) { From d9e4bd25940d62cc3fe2eb5bb3241d4938ef9df8 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 18 Nov 2020 21:32:33 +0300 Subject: [PATCH 15/40] Draft implementation of ShowPartitionsExec --- .../datasources/v2/ShowPartitionsExec.scala | 29 +++++++++++---- .../command/v2/ShowPartitionsSuite.scala | 36 +++++++++---------- 2 files changed, 41 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 9665041f8088..b584e0c25d7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.connector.catalog.{SupportsPartitionManagement, TableCatalog} import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.types.StringType +import org.apache.spark.unsafe.types.UTF8String /** * Physical plan node for showing partitions. @@ -38,10 +41,24 @@ case class ShowPartitionsExec( s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${table.name()}") } - // table.listPartitionIdentifiers(spec.get.spec) - - // scalastyle:off - throw new NotImplementedError("SHOW PARTITIONS is not implemented") - // scalastyle:on + // FIXME: handle `spec` == None + val partitionIdentifiers = table.listPartitionIdentifiers(spec.get.spec) + // Converting partition identifiers as `InternalRow` of partition values, + // for instance InternalRow(value0, value1, ..., valueN), to `InternalRow`s + // with a string in the format: "col0=value0/col1=value1/.../colN=valueN". + val schema = table.partitionSchema() + val len = schema.length + val partitions = new Array[String](len) + partitionIdentifiers.map { row => + var i = 0 + while (i < len) { + val dataType = schema(i).dataType + val partValue = row.get(i, dataType) + val partValueStr = Cast(Literal(partValue, dataType), StringType).eval().toString + partitions(i) = escapePathName(schema(i).name) + "=" + escapePathName(partValueStr) + i += 1 + } + InternalRow(UTF8String.fromString(partitions.mkString("/"))) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 35a8555a7e9c..d9d16a838a29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.SparkConf +import org.apache.spark.sql.Row import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog import org.apache.spark.sql.execution.command import org.apache.spark.sql.test.SharedSparkSession @@ -31,25 +32,24 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa override def sparkConf: SparkConf = super.sparkConf .set(s"spark.sql.catalog.$catalog", classOf[InMemoryPartitionTableCatalog].getName) - // TODO(SPARK-33452): Create a V2 SHOW PARTITIONS execution node - test("not supported SHOW PARTITIONS") { - def testV1Command(sqlCommand: String, sqlParams: String): Unit = { - val e = intercept[NotImplementedError] { - sql(s"$sqlCommand $sqlParams") - } - assert(e.getMessage.contains(s"SHOW PARTITIONS is not implemented")) - } - val t = s"$catalog.ns1.ns2.tbl" - withTable(t) { - sql( - s""" - |CREATE TABLE $t (id bigint, data string) - |$defaultUsing - |PARTITIONED BY (id) - """.stripMargin) + private def createDateTable(table: String): Unit = { + sql(s""" + |CREATE TABLE $table (price int, qty int, year int, month int) + |$defaultUsing + |partitioned by (year, month)""".stripMargin) + sql(s"ALTER TABLE $table ADD PARTITION(year = 2015, month = 1)") + sql(s"ALTER TABLE $table ADD PARTITION(year = 2015, month = 2)") + sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 2)") + sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 3)") + } - testV1Command("SHOW PARTITIONS", t) - testV1Command("SHOW PARTITIONS", s"$t PARTITION(id='1')") + test("filter by partitions") { + val table = s"$catalog.dateTable" + withTable(table) { + createDateTable(table) + checkAnswer( + sql(s"show partitions $table PARTITION(year=2015, month=1)"), + Row("year=2015/month=1") :: Nil) } } } From 761966b5a01293f7ada915ebd62994b55b38310a Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 10:35:18 +0300 Subject: [PATCH 16/40] Handle None as partitionSpec --- .../datasources/v2/ShowPartitionsExec.scala | 8 ++-- .../command/v2/ShowPartitionsSuite.scala | 37 +++++++++++++++++++ 2 files changed, 42 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index b584e0c25d7a..a4235e511931 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -34,15 +34,17 @@ case class ShowPartitionsExec( output: Seq[Attribute], catalog: TableCatalog, table: SupportsPartitionManagement, - spec: Option[ResolvedPartitionSpec]) extends V2CommandExec with LeafExecNode { + partitionSpec: Option[ResolvedPartitionSpec]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { if (table.partitioning.isEmpty) { throw new AnalysisException( s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${table.name()}") } - // FIXME: handle `spec` == None - val partitionIdentifiers = table.listPartitionIdentifiers(spec.get.spec) + // Empty `InternalRow` without any partition values is supposed to be the prefix of all + // partitions, and listPartitionIdentifiers() should return all partitions in that case. + val spec = partitionSpec.map(_.spec).getOrElse(InternalRow.empty) + val partitionIdentifiers = table.listPartitionIdentifiers(spec) // Converting partition identifiers as `InternalRow` of partition values, // for instance InternalRow(value0, value1, ..., valueN), to `InternalRow`s // with a string in the format: "col0=value0/col1=value1/.../colN=valueN". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index d9d16a838a29..ff85b1be4f6e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -43,6 +43,19 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 3)") } + test("show everything") { + val table = s"$catalog.dateTable" + withTable(table) { + createDateTable(table) + checkAnswer( + sql(s"show partitions $table"), + Row("year=2015/month=1") :: + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: + Row("year=2016/month=3") :: Nil) + } + } + test("filter by partitions") { val table = s"$catalog.dateTable" withTable(table) { @@ -52,4 +65,28 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa Row("year=2015/month=1") :: Nil) } } + + test("show everything more than 5 part keys") { + val table = s"$catalog.wideTable" + withTable(table) { + sql(s""" + |CREATE TABLE $table ( + | price int, qty int, + | year int, month int, hour int, minute int, sec int, extra int) + |$defaultUsing + |PARTITIONED BY (year, month, hour, minute, sec, extra)""".stripMargin) + sql(s""" + |ALTER TABLE $table + |ADD PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) + """.stripMargin) + sql(s""" + |ALTER TABLE $table + |ADD PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) + """.stripMargin) + checkAnswer( + sql(s"show partitions $table"), + Row("year=2016/month=3/hour=10/minute=10/sec=10/extra=1") :: + Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) + } + } } From db4567725eff0bf2251dac2dea766007a27fe041 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 14:48:13 +0300 Subject: [PATCH 17/40] Move view check to CheckAnalysis --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 10 ++++++++++ .../datasources/v2/DataSourceV2Strategy.scala | 2 -- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 452ba80b2344..60b7811d8752 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -571,6 +571,8 @@ trait CheckAnalysis extends PredicateHelper { case AlterTableDropPartition(ResolvedTable(_, _, table), parts, _, _, _) => checkAlterTablePartition(table, parts) + case showPartitions: ShowPartitions => checkShowPartitions(showPartitions) + case _ => // Fallbacks to the following checks } @@ -1003,4 +1005,12 @@ trait CheckAnalysis extends PredicateHelper { case _ => } } + + // Make sure that the `SHOW PARTITIONS` command is allowed for the table + private def checkShowPartitions(showPartitions: ShowPartitions): Unit = showPartitions match { + case ShowPartitions(v: ResolvedView, _) => + failAnalysis(s"SHOW PARTITIONS is not allowed on a view: ${v.identifier}") + + case _ => + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 0c3b9fc510bc..163037017d00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -299,8 +299,6 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case ShowCreateTable(_: ResolvedTable, _) => throw new AnalysisException("SHOW CREATE TABLE is not supported for v2 tables.") - case ShowPartitions(v: ResolvedView, _) => - throw new AnalysisException(s"SHOW PARTITIONS is not allowed on a view: ${v.identifier}") case r @ ShowPartitions( ResolvedTable(catalog, _, table: SupportsPartitionManagement), pattern @ (None | Some(ResolvedPartitionSpec(_, _)))) => From 5b15dc3cf75ac1bec92243f6974d8fbf23bc793b Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 15:14:24 +0300 Subject: [PATCH 18/40] Check that a table supports partitioning --- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 +++++- .../command/v2/ShowPartitionsSuite.scala | 19 +++++++++++++++++-- 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 60b7811d8752..0ced416d7d21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1010,7 +1010,11 @@ trait CheckAnalysis extends PredicateHelper { private def checkShowPartitions(showPartitions: ShowPartitions): Unit = showPartitions match { case ShowPartitions(v: ResolvedView, _) => failAnalysis(s"SHOW PARTITIONS is not allowed on a view: ${v.identifier}") - + case ShowPartitions(child, _) if !child.isInstanceOf[ResolvedTable] => + failAnalysis(s"Cannot resolve the table for SHOW PARTITIONS") + case ShowPartitions(rt: ResolvedTable, _) + if !rt.table.isInstanceOf[SupportsPartitionManagement] => + failAnalysis(s"SHOW PARTITIONS cannot run for a table which does not support partitioning") case _ => } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index ff85b1be4f6e..5cf223af3687 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.SparkConf -import org.apache.spark.sql.Row -import org.apache.spark.sql.connector.InMemoryPartitionTableCatalog +import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.connector.{InMemoryPartitionTableCatalog, InMemoryTableCatalog} import org.apache.spark.sql.execution.command import org.apache.spark.sql.test.SharedSparkSession @@ -31,6 +31,7 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa override def sparkConf: SparkConf = super.sparkConf .set(s"spark.sql.catalog.$catalog", classOf[InMemoryPartitionTableCatalog].getName) + .set(s"spark.sql.catalog.non_part_$catalog", classOf[InMemoryTableCatalog].getName) private def createDateTable(table: String): Unit = { sql(s""" @@ -43,6 +44,20 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 3)") } + test("a table does not support partitioning") { + val table = s"non_part_$catalog.tab1" + withTable(table) { + sql(s""" + |CREATE TABLE $table (price int, qty int, year int, month int) + |$defaultUsing""".stripMargin) + val errMsg = intercept[AnalysisException] { + sql(s"SHOW PARTITIONS $table") + }.getMessage + assert(errMsg.contains( + "SHOW PARTITIONS cannot run for a table which does not support partitioning")) + } + } + test("show everything") { val table = s"$catalog.dateTable" withTable(table) { From f6b68147852bf2bae0256b1d182673c16766598a Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 15:21:15 +0300 Subject: [PATCH 19/40] Remove wrong check from ShowPartitionsExec --- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ++++ .../sql/execution/datasources/v2/ShowPartitionsExec.scala | 5 ----- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 0ced416d7d21..428b7fede275 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1015,6 +1015,10 @@ trait CheckAnalysis extends PredicateHelper { case ShowPartitions(rt: ResolvedTable, _) if !rt.table.isInstanceOf[SupportsPartitionManagement] => failAnalysis(s"SHOW PARTITIONS cannot run for a table which does not support partitioning") + case ShowPartitions(ResolvedTable(_, _, partTable: SupportsPartitionManagement), _) + if partTable.partitionSchema().isEmpty => + failAnalysis( + s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${partTable.name()}") case _ => } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index a4235e511931..55f5f339fe15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -36,11 +36,6 @@ case class ShowPartitionsExec( table: SupportsPartitionManagement, partitionSpec: Option[ResolvedPartitionSpec]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { - if (table.partitioning.isEmpty) { - throw new AnalysisException( - s"SHOW PARTITIONS is not allowed on a table that is not partitioned: ${table.name()}") - } - // Empty `InternalRow` without any partition values is supposed to be the prefix of all // partitions, and listPartitionIdentifiers() should return all partitions in that case. val spec = partitionSpec.map(_.spec).getOrElse(InternalRow.empty) From 8c59e8ae53a383a5eb79ba127bf402485c8e7f9e Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 15:47:19 +0300 Subject: [PATCH 20/40] Add v2 test for non-partitioning columns --- .../execution/command/v2/ShowPartitionsSuite.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 5cf223af3687..56a0ce1e631d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -104,4 +104,15 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) } } + + test("non-partitioning columns") { + val table = s"$catalog.dateTable" + withTable(table) { + createDateTable(table) + val errMsg = intercept[AnalysisException] { + sql(s"SHOW PARTITIONS $table PARTITION(abcd=2015, xyz=1)") + }.getMessage + assert(errMsg.contains("Partition key abcd,xyz not exists")) + } + } } From b62ed58eb7935a022cf52c2b01cc733dad7ed203 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 16:00:45 +0300 Subject: [PATCH 21/40] Move the test "non-partitioning columns" to the common trait --- .../command/ShowPartitionsSuiteBase.scala | 16 ++++++++++++++++ .../command/v1/ShowPartitionsSuite.scala | 17 +++++------------ .../command/v2/ShowPartitionsSuite.scala | 17 +++++------------ 3 files changed, 26 insertions(+), 24 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 9e6a16894b80..14dbc4993644 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -28,6 +28,8 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { protected def catalog: String protected def defaultNamespace: Seq[String] protected def defaultUsing: String + protected def createDateTable(table: String): Unit + protected def wrongPartitionColumnsError(columns: String*): String override def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -47,4 +49,18 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { } } } + + test("non-partitioning columns") { + withNamespace(s"$catalog.ns") { + sql(s"CREATE NAMESPACE $catalog.ns") + val table = s"$catalog.ns.dateTable" + withTable(table) { + createDateTable(table) + val errMsg = intercept[AnalysisException] { + sql(s"SHOW PARTITIONS $table PARTITION(abcd=2015, xyz=1)") + }.getMessage + assert(errMsg.contains(wrongPartitionColumnsError("abcd", "xyz"))) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index e57b481aa4c8..5882cf0eb8cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -29,7 +29,7 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { override def defaultNamespace: Seq[String] = Seq("default") override def defaultUsing: String = "USING parquet" - private def createDateTable(table: String): Unit = { + override protected def createDateTable(table: String): Unit = { sql(s""" |CREATE TABLE $table (price int, qty int, year int, month int) |$defaultUsing @@ -40,6 +40,10 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { sql(s"INSERT INTO $table PARTITION(year = 2016, month = 3) SELECT 3, 3") } + override protected def wrongPartitionColumnsError(columns: String*): String = { + s"Non-partitioning column(s) ${columns.mkString("[", ", ", "]")} are specified" + } + test("show everything") { val table = "dateTable" withTable(table) { @@ -102,17 +106,6 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { } } - test("non-partitioning columns") { - val table = "dateTable" - withTable(table) { - createDateTable(table) - val errMsg = intercept[AnalysisException] { - sql(s"SHOW PARTITIONS $table PARTITION(abcd=2015, xyz=1)") - }.getMessage - assert(errMsg.contains("Non-partitioning column(s) [abcd, xyz] are specified")) - } - } - // The test fails for V2 Table Catalogs with the exception: // org.apache.spark.sql.AnalysisException: CREATE VIEW is only supported with v1 tables. test("show partitions of a view") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 56a0ce1e631d..198c9dd5636e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -33,7 +33,7 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa .set(s"spark.sql.catalog.$catalog", classOf[InMemoryPartitionTableCatalog].getName) .set(s"spark.sql.catalog.non_part_$catalog", classOf[InMemoryTableCatalog].getName) - private def createDateTable(table: String): Unit = { + override protected def createDateTable(table: String): Unit = { sql(s""" |CREATE TABLE $table (price int, qty int, year int, month int) |$defaultUsing @@ -44,6 +44,10 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 3)") } + override protected def wrongPartitionColumnsError(columns: String*): String = { + s"Partition key ${columns.mkString(",")} not exists" + } + test("a table does not support partitioning") { val table = s"non_part_$catalog.tab1" withTable(table) { @@ -104,15 +108,4 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) } } - - test("non-partitioning columns") { - val table = s"$catalog.dateTable" - withTable(table) { - createDateTable(table) - val errMsg = intercept[AnalysisException] { - sql(s"SHOW PARTITIONS $table PARTITION(abcd=2015, xyz=1)") - }.getMessage - assert(errMsg.contains("Partition key abcd,xyz not exists")) - } - } } From 6d14c568d09abc08ba73e1356ac85babfdd4c427 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 16:17:25 +0300 Subject: [PATCH 22/40] Move a positive test to the common trait --- .../command/ShowPartitionsSuiteBase.scala | 25 ++++++++++++++++++- .../command/v1/ShowPartitionsSuite.scala | 9 +------ .../command/v2/ShowPartitionsSuite.scala | 13 ---------- 3 files changed, 25 insertions(+), 22 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 14dbc4993644..a71f9f4a0c43 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.execution.command import org.scalactic.source.Position import org.scalatest.Tag -import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.{StringType, StructType} trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { protected def version: String @@ -31,6 +32,12 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { protected def createDateTable(table: String): Unit protected def wrongPartitionColumnsError(columns: String*): String + protected def runShowPartitionsSql(sqlText: String, expected: Seq[Row]): Unit = { + val df = spark.sql(sqlText) + assert(df.schema === new StructType().add("partition", StringType, false)) + checkAnswer(df, expected) + } + override def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { super.test(s"SHOW PARTITIONS $version: " + testName, testTags: _*)(testFun) @@ -63,4 +70,20 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { } } } + + test("show everything") { + withNamespace(s"$catalog.ns") { + sql(s"CREATE NAMESPACE $catalog.ns") + val table = s"$catalog.ns.dateTable" + withTable(table) { + createDateTable(table) + runShowPartitionsSql( + s"show partitions $table", + Row("year=2015/month=1") :: + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: + Row("year=2016/month=3") :: Nil) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 5882cf0eb8cf..5e23e960a7ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -44,17 +44,10 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { s"Non-partitioning column(s) ${columns.mkString("[", ", ", "]")} are specified" } - test("show everything") { + test("show everything in the default database") { val table = "dateTable" withTable(table) { createDateTable(table) - checkAnswer( - sql(s"show partitions $table"), - Row("year=2015/month=1") :: - Row("year=2015/month=2") :: - Row("year=2016/month=2") :: - Row("year=2016/month=3") :: Nil) - checkAnswer( sql(s"show partitions default.$table"), Row("year=2015/month=1") :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 198c9dd5636e..b2b617ba4285 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -62,19 +62,6 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa } } - test("show everything") { - val table = s"$catalog.dateTable" - withTable(table) { - createDateTable(table) - checkAnswer( - sql(s"show partitions $table"), - Row("year=2015/month=1") :: - Row("year=2015/month=2") :: - Row("year=2016/month=2") :: - Row("year=2016/month=3") :: Nil) - } - } - test("filter by partitions") { val table = s"$catalog.dateTable" withTable(table) { From b800229fa4154fb3cfc33e8395c9a34ace3296a5 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 16:23:24 +0300 Subject: [PATCH 23/40] Use runShowPartitionsSql() --- .../command/ShowPartitionsSuiteBase.scala | 5 ++-- .../command/v1/ShowPartitionsSuite.scala | 24 +++++++++---------- .../command/v2/ShowPartitionsSuite.scala | 8 +++---- 3 files changed, 19 insertions(+), 18 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index a71f9f4a0c43..67b98da44263 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -31,10 +31,11 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { protected def defaultUsing: String protected def createDateTable(table: String): Unit protected def wrongPartitionColumnsError(columns: String*): String - + // Gets the schema of `SHOW PARTITIONS` + private val showSchema: StructType = new StructType().add("partition", StringType, false) protected def runShowPartitionsSql(sqlText: String, expected: Seq[Row]): Unit = { val df = spark.sql(sqlText) - assert(df.schema === new StructType().add("partition", StringType, false)) + assert(df.schema === showSchema) checkAnswer(df, expected) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 5e23e960a7ad..df132dc8ffde 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -48,8 +48,8 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { val table = "dateTable" withTable(table) { createDateTable(table) - checkAnswer( - sql(s"show partitions default.$table"), + runShowPartitionsSql( + s"show partitions default.$table", Row("year=2015/month=1") :: Row("year=2015/month=2") :: Row("year=2016/month=2") :: @@ -61,15 +61,15 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { val table = "dateTable" withTable(table) { createDateTable(table) - checkAnswer( - sql(s"show partitions default.$table PARTITION(year=2015)"), + runShowPartitionsSql( + s"show partitions default.$table PARTITION(year=2015)", Row("year=2015/month=1") :: Row("year=2015/month=2") :: Nil) - checkAnswer( - sql(s"show partitions default.$table PARTITION(year=2015, month=1)"), + runShowPartitionsSql( + s"show partitions default.$table PARTITION(year=2015, month=1)", Row("year=2015/month=1") :: Nil) - checkAnswer( - sql(s"show partitions default.$table PARTITION(month=2)"), + runShowPartitionsSql( + s"show partitions default.$table PARTITION(month=2)", Row("year=2015/month=2") :: Row("year=2016/month=2") :: Nil) } @@ -92,8 +92,8 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { |INSERT INTO $table |PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 """.stripMargin) - checkAnswer( - sql(s"show partitions $table"), + runShowPartitionsSql( + s"show partitions $table", Row("year=2016/month=3/hour=10/minute=10/sec=10/extra=1") :: Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) } @@ -136,8 +136,8 @@ class ShowPartitionsSuite extends ShowPartitionsSuiteBase with SharedSparkSessio val viewName = "test_view" withTempView(viewName) { sql(s""" - |CREATE TEMPORARY VIEW $viewName (c1 INT, c2 STRING) - |$defaultUsing""".stripMargin) + |CREATE TEMPORARY VIEW $viewName (c1 INT, c2 STRING) + |$defaultUsing""".stripMargin) val errMsg = intercept[NoSuchTableException] { sql(s"SHOW PARTITIONS $viewName") }.getMessage diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index b2b617ba4285..9e1a19a638e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -66,8 +66,8 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa val table = s"$catalog.dateTable" withTable(table) { createDateTable(table) - checkAnswer( - sql(s"show partitions $table PARTITION(year=2015, month=1)"), + runShowPartitionsSql( + s"show partitions $table PARTITION(year=2015, month=1)", Row("year=2015/month=1") :: Nil) } } @@ -89,8 +89,8 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa |ALTER TABLE $table |ADD PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) """.stripMargin) - checkAnswer( - sql(s"show partitions $table"), + runShowPartitionsSql( + s"show partitions $table", Row("year=2016/month=3/hour=10/minute=10/sec=10/extra=1") :: Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) } From ac0d2f166e4d6d3bd58dc022d4a6da9ef1f48f19 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 16:42:48 +0300 Subject: [PATCH 24/40] Move checking by all partitioned columns to the common trait --- .../execution/command/ShowPartitionsSuiteBase.scala | 13 +++++++++++++ .../execution/command/v1/ShowPartitionsSuite.scala | 3 --- .../execution/command/v2/ShowPartitionsSuite.scala | 10 ---------- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 67b98da44263..dc5db4a41e55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -87,4 +87,17 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { } } } + + test("filter by partitions with all partitioned columns") { + withNamespace(s"$catalog.ns") { + sql(s"CREATE NAMESPACE $catalog.ns") + val table = s"$catalog.ns.dateTable" + withTable(table) { + createDateTable(table) + runShowPartitionsSql( + s"show partitions $table PARTITION(year=2015, month=1)", + Row("year=2015/month=1") :: Nil) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index df132dc8ffde..5aa1d8dd70a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -65,9 +65,6 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { s"show partitions default.$table PARTITION(year=2015)", Row("year=2015/month=1") :: Row("year=2015/month=2") :: Nil) - runShowPartitionsSql( - s"show partitions default.$table PARTITION(year=2015, month=1)", - Row("year=2015/month=1") :: Nil) runShowPartitionsSql( s"show partitions default.$table PARTITION(month=2)", Row("year=2015/month=2") :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 9e1a19a638e5..939707f35c75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -62,16 +62,6 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa } } - test("filter by partitions") { - val table = s"$catalog.dateTable" - withTable(table) { - createDateTable(table) - runShowPartitionsSql( - s"show partitions $table PARTITION(year=2015, month=1)", - Row("year=2015/month=1") :: Nil) - } - } - test("show everything more than 5 part keys") { val table = s"$catalog.wideTable" withTable(table) { From 18cdaf6601da6e343a167bfcd1bbc1831dcc90cc Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 16:52:09 +0300 Subject: [PATCH 25/40] Move the test "show everything more than 5 part keys" to the common trait ShowPartitionsSuiteBase --- .../command/ShowPartitionsSuiteBase.scala | 16 ++++++++ .../command/v1/ShowPartitionsSuite.scala | 41 ++++++++----------- .../command/v2/ShowPartitionsSuite.scala | 41 ++++++++----------- 3 files changed, 50 insertions(+), 48 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index dc5db4a41e55..b195b83ad127 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -30,6 +30,7 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { protected def defaultNamespace: Seq[String] protected def defaultUsing: String protected def createDateTable(table: String): Unit + protected def createWideTable(table: String): Unit protected def wrongPartitionColumnsError(columns: String*): String // Gets the schema of `SHOW PARTITIONS` private val showSchema: StructType = new StructType().add("partition", StringType, false) @@ -100,4 +101,19 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { } } } + + + test("show everything more than 5 part keys") { + withNamespace(s"$catalog.ns") { + sql(s"CREATE NAMESPACE $catalog.ns") + val table = s"$catalog.ns.wideTable" + withTable(table) { + createWideTable(table) + runShowPartitionsSql( + s"show partitions $table", + Row("year=2016/month=3/hour=10/minute=10/sec=10/extra=1") :: + Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 5aa1d8dd70a6..c01c3c8d6f34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -40,6 +40,23 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { sql(s"INSERT INTO $table PARTITION(year = 2016, month = 3) SELECT 3, 3") } + override protected def createWideTable(table: String): Unit = { + sql(s""" + |CREATE TABLE $table ( + | price int, qty int, + | year int, month int, hour int, minute int, sec int, extra int) + |$defaultUsing + |PARTITIONED BY (year, month, hour, minute, sec, extra)""".stripMargin) + sql(s""" + |INSERT INTO $table + |PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 + """.stripMargin) + sql(s""" + |INSERT INTO $table + |PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 + """.stripMargin) + } + override protected def wrongPartitionColumnsError(columns: String*): String = { s"Non-partitioning column(s) ${columns.mkString("[", ", ", "]")} are specified" } @@ -72,30 +89,6 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { } } - test("show everything more than 5 part keys") { - val table = "wideTable" - withTable(table) { - sql(s""" - |CREATE TABLE $table ( - | price int, qty int, - | year int, month int, hour int, minute int, sec int, extra int) - |$defaultUsing - |PARTITIONED BY (year, month, hour, minute, sec, extra)""".stripMargin) - sql(s""" - |INSERT INTO $table - |PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 - """.stripMargin) - sql(s""" - |INSERT INTO $table - |PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 - """.stripMargin) - runShowPartitionsSql( - s"show partitions $table", - Row("year=2016/month=3/hour=10/minute=10/sec=10/extra=1") :: - Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) - } - } - // The test fails for V2 Table Catalogs with the exception: // org.apache.spark.sql.AnalysisException: CREATE VIEW is only supported with v1 tables. test("show partitions of a view") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 939707f35c75..85df10430e63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -44,6 +44,23 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 3)") } + override protected def createWideTable(table: String): Unit = { + sql(s""" + |CREATE TABLE $table ( + | price int, qty int, + | year int, month int, hour int, minute int, sec int, extra int) + |$defaultUsing + |PARTITIONED BY (year, month, hour, minute, sec, extra)""".stripMargin) + sql(s""" + |ALTER TABLE $table + |ADD PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) + """.stripMargin) + sql(s""" + |ALTER TABLE $table + |ADD PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) + """.stripMargin) + } + override protected def wrongPartitionColumnsError(columns: String*): String = { s"Partition key ${columns.mkString(",")} not exists" } @@ -61,28 +78,4 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa "SHOW PARTITIONS cannot run for a table which does not support partitioning")) } } - - test("show everything more than 5 part keys") { - val table = s"$catalog.wideTable" - withTable(table) { - sql(s""" - |CREATE TABLE $table ( - | price int, qty int, - | year int, month int, hour int, minute int, sec int, extra int) - |$defaultUsing - |PARTITIONED BY (year, month, hour, minute, sec, extra)""".stripMargin) - sql(s""" - |ALTER TABLE $table - |ADD PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) - """.stripMargin) - sql(s""" - |ALTER TABLE $table - |ADD PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) - """.stripMargin) - runShowPartitionsSql( - s"show partitions $table", - Row("year=2016/month=3/hour=10/minute=10/sec=10/extra=1") :: - Row("year=2016/month=4/hour=10/minute=10/sec=10/extra=1") :: Nil) - } - } } From a08177e520dda8e57b3cc7c1143052ecddf89fa0 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 18:48:06 +0300 Subject: [PATCH 26/40] Remove unused imports --- .../sql/execution/datasources/v2/DataSourceV2Strategy.scala | 2 +- .../spark/sql/execution/datasources/v2/ShowPartitionsExec.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 163037017d00..8c2154666301 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy} -import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable, ResolvedView} +import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable} import org.apache.spark.sql.catalyst.expressions.{And, Expression, NamedExpression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 55f5f339fe15..f7f1911a5d0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName From 797da1749e59211b2b051295d1d67333aeda0dfd Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 19:11:16 +0300 Subject: [PATCH 27/40] Remove an unused import --- .../spark/sql/execution/command/v2/ShowPartitionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 85df10430e63..eaa03ddc0cdc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.command.v2 import org.apache.spark.SparkConf -import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.connector.{InMemoryPartitionTableCatalog, InMemoryTableCatalog} import org.apache.spark.sql.execution.command import org.apache.spark.sql.test.SharedSparkSession From c1faacb09030a788ff25045ba82de340383c0d10 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 19:13:03 +0300 Subject: [PATCH 28/40] Fix indentations --- .../execution/command/ShowPartitionsSuiteBase.scala | 6 +++--- .../sql/execution/command/v1/ShowPartitionsSuite.scala | 10 +++++----- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index b195b83ad127..f533255314a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -82,9 +82,9 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { runShowPartitionsSql( s"show partitions $table", Row("year=2015/month=1") :: - Row("year=2015/month=2") :: - Row("year=2016/month=2") :: - Row("year=2016/month=3") :: Nil) + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: + Row("year=2016/month=3") :: Nil) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index c01c3c8d6f34..36c35945230c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -68,9 +68,9 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { runShowPartitionsSql( s"show partitions default.$table", Row("year=2015/month=1") :: - Row("year=2015/month=2") :: - Row("year=2016/month=2") :: - Row("year=2016/month=3") :: Nil) + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: + Row("year=2016/month=3") :: Nil) } } @@ -81,11 +81,11 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { runShowPartitionsSql( s"show partitions default.$table PARTITION(year=2015)", Row("year=2015/month=1") :: - Row("year=2015/month=2") :: Nil) + Row("year=2015/month=2") :: Nil) runShowPartitionsSql( s"show partitions default.$table PARTITION(month=2)", Row("year=2015/month=2") :: - Row("year=2016/month=2") :: Nil) + Row("year=2016/month=2") :: Nil) } } From 0d8738c9b60c69fb0b894b41a329e46bfa2b6e9f Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 19 Nov 2020 20:38:30 +0300 Subject: [PATCH 29/40] Fix PartitionedTablePerfStatsSuite --- .../sql/hive/PartitionedTablePerfStatsSuite.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala index 62db0f86110d..49e26614e13c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala @@ -283,8 +283,7 @@ class PartitionedTablePerfStatsSuite } } - // FIXME: SPARK-33452 - ignore("hive table: num hive client calls does not scale with partition count") { + test("hive table: num hive client calls does not scale with partition count") { withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { withTable("test") { withTempDir { dir => @@ -301,14 +300,13 @@ class PartitionedTablePerfStatsSuite HiveCatalogMetrics.reset() assert(spark.sql("show partitions test").count() == 100) - assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() < 10) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() <= 10) } } } } - // FIXME: SPARK-33452 - ignore("datasource table: num hive client calls does not scale with partition count") { + test("datasource table: num hive client calls does not scale with partition count") { withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { withTable("test") { withTempDir { dir => @@ -325,7 +323,7 @@ class PartitionedTablePerfStatsSuite HiveCatalogMetrics.reset() assert(spark.sql("show partitions test").count() == 100) - assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() < 10) + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount() <= 10) } } } From c34995fccb4799be29b90b1ecb6eba6e85cdb98d Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Fri, 20 Nov 2020 11:31:15 +0300 Subject: [PATCH 30/40] Pass time zone id to Cast --- .../sql/execution/datasources/v2/ShowPartitionsExec.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index f7f1911a5d0d..3a7a587ebb9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.connector.catalog.{SupportsPartitionManagement, TableCatalog} import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StringType import org.apache.spark.unsafe.types.UTF8String @@ -45,12 +46,14 @@ case class ShowPartitionsExec( val schema = table.partitionSchema() val len = schema.length val partitions = new Array[String](len) + val timeZoneId = SQLConf.get.sessionLocalTimeZone partitionIdentifiers.map { row => var i = 0 while (i < len) { val dataType = schema(i).dataType val partValue = row.get(i, dataType) - val partValueStr = Cast(Literal(partValue, dataType), StringType).eval().toString + val partValueStr = Cast(Literal(partValue, dataType), StringType, Some(timeZoneId)) + .eval().toString partitions(i) = escapePathName(schema(i).name) + "=" + escapePathName(partValueStr) i += 1 } From feffb90dd60a274dce4a3b4ec3da0c2b6c2dcc50 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sat, 21 Nov 2020 11:42:27 +0300 Subject: [PATCH 31/40] Move table creation to the common trait --- .../command/ShowPartitionsSuiteBase.scala | 32 +++++++++++++++++-- .../command/v1/ShowPartitionsSuite.scala | 28 ---------------- .../command/v2/ShowPartitionsSuite.scala | 28 ---------------- 3 files changed, 30 insertions(+), 58 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index f533255314a6..a66dadb3719e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -29,8 +29,6 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { protected def catalog: String protected def defaultNamespace: Seq[String] protected def defaultUsing: String - protected def createDateTable(table: String): Unit - protected def createWideTable(table: String): Unit protected def wrongPartitionColumnsError(columns: String*): String // Gets the schema of `SHOW PARTITIONS` private val showSchema: StructType = new StructType().add("partition", StringType, false) @@ -45,6 +43,36 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { super.test(s"SHOW PARTITIONS $version: " + testName, testTags: _*)(testFun) } + + protected def createDateTable(table: String): Unit = { + sql(s""" + |CREATE TABLE $table (price int, qty int, year int, month int) + |$defaultUsing + |partitioned by (year, month)""".stripMargin) + sql(s"INSERT INTO $table PARTITION(year = 2015, month = 1) SELECT 1, 1") + sql(s"INSERT INTO $table PARTITION(year = 2015, month = 2) SELECT 2, 2") + sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 2)") + sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 3)") + } + + protected def createWideTable(table: String): Unit = { + sql(s""" + |CREATE TABLE $table ( + | price int, qty int, + | year int, month int, hour int, minute int, sec int, extra int) + |$defaultUsing + |PARTITIONED BY (year, month, hour, minute, sec, extra) + |""".stripMargin) + sql(s""" + |INSERT INTO $table + |PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 + |""".stripMargin) + sql(s""" + |ALTER TABLE $table + |ADD PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) + |""".stripMargin) + } + test("show partitions of non-partitioned table") { withNamespace(s"$catalog.ns") { sql(s"CREATE NAMESPACE $catalog.ns") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 36c35945230c..70ebbfa72507 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -29,34 +29,6 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { override def defaultNamespace: Seq[String] = Seq("default") override def defaultUsing: String = "USING parquet" - override protected def createDateTable(table: String): Unit = { - sql(s""" - |CREATE TABLE $table (price int, qty int, year int, month int) - |$defaultUsing - |partitioned by (year, month)""".stripMargin) - sql(s"INSERT INTO $table PARTITION(year = 2015, month = 1) SELECT 1, 1") - sql(s"INSERT INTO $table PARTITION(year = 2015, month = 2) SELECT 2, 2") - sql(s"INSERT INTO $table PARTITION(year = 2016, month = 2) SELECT 3, 3") - sql(s"INSERT INTO $table PARTITION(year = 2016, month = 3) SELECT 3, 3") - } - - override protected def createWideTable(table: String): Unit = { - sql(s""" - |CREATE TABLE $table ( - | price int, qty int, - | year int, month int, hour int, minute int, sec int, extra int) - |$defaultUsing - |PARTITIONED BY (year, month, hour, minute, sec, extra)""".stripMargin) - sql(s""" - |INSERT INTO $table - |PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 - """.stripMargin) - sql(s""" - |INSERT INTO $table - |PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) SELECT 3, 3 - """.stripMargin) - } - override protected def wrongPartitionColumnsError(columns: String*): String = { s"Non-partitioning column(s) ${columns.mkString("[", ", ", "]")} are specified" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index eaa03ddc0cdc..1a76f2e564f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -33,34 +33,6 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa .set(s"spark.sql.catalog.$catalog", classOf[InMemoryPartitionTableCatalog].getName) .set(s"spark.sql.catalog.non_part_$catalog", classOf[InMemoryTableCatalog].getName) - override protected def createDateTable(table: String): Unit = { - sql(s""" - |CREATE TABLE $table (price int, qty int, year int, month int) - |$defaultUsing - |partitioned by (year, month)""".stripMargin) - sql(s"ALTER TABLE $table ADD PARTITION(year = 2015, month = 1)") - sql(s"ALTER TABLE $table ADD PARTITION(year = 2015, month = 2)") - sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 2)") - sql(s"ALTER TABLE $table ADD PARTITION(year = 2016, month = 3)") - } - - override protected def createWideTable(table: String): Unit = { - sql(s""" - |CREATE TABLE $table ( - | price int, qty int, - | year int, month int, hour int, minute int, sec int, extra int) - |$defaultUsing - |PARTITIONED BY (year, month, hour, minute, sec, extra)""".stripMargin) - sql(s""" - |ALTER TABLE $table - |ADD PARTITION(year = 2016, month = 3, hour = 10, minute = 10, sec = 10, extra = 1) - """.stripMargin) - sql(s""" - |ALTER TABLE $table - |ADD PARTITION(year = 2016, month = 4, hour = 10, minute = 10, sec = 10, extra = 1) - """.stripMargin) - } - override protected def wrongPartitionColumnsError(columns: String*): String = { s"Partition key ${columns.mkString(",")} not exists" } From bf5154aaee83975ad95b5057812d005bdf017864 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sat, 21 Nov 2020 11:45:24 +0300 Subject: [PATCH 32/40] Refactoring: remove unused `defaultNamespace` --- .../spark/sql/execution/command/ShowPartitionsSuiteBase.scala | 2 -- .../spark/sql/execution/command/v1/ShowPartitionsSuite.scala | 1 - .../spark/sql/execution/command/v2/ShowPartitionsSuite.scala | 1 - 3 files changed, 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index a66dadb3719e..6b4f04a73756 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.types.{StringType, StructType} trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { protected def version: String protected def catalog: String - protected def defaultNamespace: Seq[String] protected def defaultUsing: String protected def wrongPartitionColumnsError(columns: String*): String // Gets the schema of `SHOW PARTITIONS` @@ -43,7 +42,6 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { super.test(s"SHOW PARTITIONS $version: " + testName, testTags: _*)(testFun) } - protected def createDateTable(table: String): Unit = { sql(s""" |CREATE TABLE $table (price int, qty int, year int, month int) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 70ebbfa72507..33d594d7b214 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.test.SharedSparkSession trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { override def version: String = "V1" override def catalog: String = CatalogManager.SESSION_CATALOG_NAME - override def defaultNamespace: Seq[String] = Seq("default") override def defaultUsing: String = "USING parquet" override protected def wrongPartitionColumnsError(columns: String*): String = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index 1a76f2e564f8..a4b991662e00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.test.SharedSparkSession class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSparkSession { override def version: String = "V2" override def catalog: String = "test_catalog" - override def defaultNamespace: Seq[String] = Nil override def defaultUsing: String = "USING _" override def sparkConf: SparkConf = super.sparkConf From 0a1f9726bda782f659a380b04f128956715e553d Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 25 Nov 2020 17:44:40 +0300 Subject: [PATCH 33/40] fix error msg --- .../spark/sql/execution/command/v2/ShowPartitionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala index a4b991662e00..ca47a713ad60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowPartitionsSuite.scala @@ -33,7 +33,7 @@ class ShowPartitionsSuite extends command.ShowPartitionsSuiteBase with SharedSpa .set(s"spark.sql.catalog.non_part_$catalog", classOf[InMemoryTableCatalog].getName) override protected def wrongPartitionColumnsError(columns: String*): String = { - s"Partition key ${columns.mkString(",")} not exists" + s"${columns.head} is not a valid partition column" } test("a table does not support partitioning") { From a4acf4060c02cfc865a8e560d8cef898353085ed Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 25 Nov 2020 17:45:08 +0300 Subject: [PATCH 34/40] fix merge conflicts --- .../spark/sql/catalyst/analysis/ResolvePartitionSpec.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index 91dcd31ba2db..aa90ec96f80a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -41,7 +41,10 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { r.copy(parts = resolvePartitionSpecs(table.name, partSpecs, table.partitionSchema())) case r @ ShowPartitions(ResolvedTable(_, _, table: SupportsPartitionManagement), partSpecs) => - r.copy(pattern = resolvePartitionSpecs(partSpecs.toSeq, table.partitionSchema()).headOption) + r.copy(pattern = resolvePartitionSpecs( + table.name, + partSpecs.toSeq, + table.partitionSchema()).headOption) } private def resolvePartitionSpecs( From 9db99233aaeb17fce65ab1525d611e48d507c5ed Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 25 Nov 2020 20:37:22 +0300 Subject: [PATCH 35/40] List by partition names --- .../analysis/ResolvePartitionSpec.scala | 25 ++++++++++--------- .../catalyst/analysis/v2ResolutionPlans.scala | 3 ++- .../v2/AlterTableAddPartitionExec.scala | 8 +++--- .../v2/AlterTableDropPartitionExec.scala | 2 +- .../datasources/v2/ShowPartitionsExec.scala | 10 +++++--- .../command/ShowPartitionsSuiteBase.scala | 11 ++++++-- .../command/v1/ShowPartitionsSuite.scala | 15 ----------- 7 files changed, 35 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index aa90ec96f80a..69997f0c2973 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -53,25 +53,26 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { partSchema: StructType): Seq[ResolvedPartitionSpec] = partSpecs.map { case unresolvedPartSpec: UnresolvedPartitionSpec => + val normalizedSpec = normalizePartitionSpec( + unresolvedPartSpec.spec, + partSchema.map(_.name), + tableName, + conf.resolver) + val partitionNames = normalizedSpec.keySet + val requestedFields = partSchema.filter(field => partitionNames.contains(field.name)) ResolvedPartitionSpec( - convertToPartIdent(tableName, unresolvedPartSpec.spec, partSchema), + requestedFields.map(_.name), + convertToPartIdent(normalizedSpec, requestedFields), unresolvedPartSpec.location) case resolvedPartitionSpec: ResolvedPartitionSpec => resolvedPartitionSpec } private def convertToPartIdent( - tableName: String, - partitionSpec: TablePartitionSpec, - partSchema: StructType): InternalRow = { - val normalizedSpec = normalizePartitionSpec( - partitionSpec, - partSchema.map(_.name), - tableName, - conf.resolver) - - val partValues = partSchema.map { part => - val raw = normalizedSpec.get(part.name).orNull + spec: TablePartitionSpec, + schema: Seq[StructField]): InternalRow = { + val partValues = schema.map { part => + val raw = spec.get(part.name).orNull Cast(Literal.create(raw, StringType), part.dataType, Some(conf.sessionLocalTimeZone)).eval() } InternalRow.fromSeq(partValues) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 0e883a88f269..e8938c50799b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -88,7 +88,8 @@ case class ResolvedTable(catalog: TableCatalog, identifier: Identifier, table: T } case class ResolvedPartitionSpec( - spec: InternalRow, + names: Seq[String], + ident: InternalRow, location: Option[String] = None) extends PartitionSpec /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableAddPartitionExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableAddPartitionExec.scala index 0171cdd9ca41..d7fe25cff206 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableAddPartitionExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableAddPartitionExec.scala @@ -37,20 +37,20 @@ case class AlterTableAddPartitionExec( override protected def run(): Seq[InternalRow] = { val (existsParts, notExistsParts) = - partSpecs.partition(p => table.partitionExists(p.spec)) + partSpecs.partition(p => table.partitionExists(p.ident)) if (existsParts.nonEmpty && !ignoreIfExists) { throw new PartitionsAlreadyExistException( - table.name(), existsParts.map(_.spec), table.partitionSchema()) + table.name(), existsParts.map(_.ident), table.partitionSchema()) } notExistsParts match { case Seq() => // Nothing will be done case Seq(partitionSpec) => val partProp = partitionSpec.location.map(loc => "location" -> loc).toMap - table.createPartition(partitionSpec.spec, partProp.asJava) + table.createPartition(partitionSpec.ident, partProp.asJava) case _ if table.isInstanceOf[SupportsAtomicPartitionManagement] => - val partIdents = notExistsParts.map(_.spec) + val partIdents = notExistsParts.map(_.ident) val partProps = notExistsParts.map(_.location.map(loc => "location" -> loc).toMap) table.asAtomicPartitionable .createPartitions( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableDropPartitionExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableDropPartitionExec.scala index 09a65804a05e..c7a68ecb2bbe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableDropPartitionExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableDropPartitionExec.scala @@ -35,7 +35,7 @@ case class AlterTableDropPartitionExec( override protected def run(): Seq[InternalRow] = { val (existsPartIdents, notExistsPartIdents) = - partSpecs.map(_.spec).partition(table.partitionExists) + partSpecs.map(_.ident).partition(table.partitionExists) if (notExistsPartIdents.nonEmpty && !ignoreIfNotExists) { throw new NoSuchPartitionsException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala index 3a7a587ebb9c..44d6f4495f55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala @@ -36,10 +36,12 @@ case class ShowPartitionsExec( table: SupportsPartitionManagement, partitionSpec: Option[ResolvedPartitionSpec]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { - // Empty `InternalRow` without any partition values is supposed to be the prefix of all - // partitions, and listPartitionIdentifiers() should return all partitions in that case. - val spec = partitionSpec.map(_.spec).getOrElse(InternalRow.empty) - val partitionIdentifiers = table.listPartitionIdentifiers(spec) + val (names, ident) = partitionSpec + .map(spec => (spec.names, spec.ident)) + // listPartitionByNames() should return all partitions if the partition spec + // does not specify any partition names. + .getOrElse((Seq.empty[String], InternalRow.empty)) + val partitionIdentifiers = table.listPartitionByNames(names.toArray, ident) // Converting partition identifiers as `InternalRow` of partition values, // for instance InternalRow(value0, value1, ..., valueN), to `InternalRow`s // with a string in the format: "col0=value0/col1=value1/.../colN=valueN". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala index 6b4f04a73756..82457f96a300 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsSuiteBase.scala @@ -115,20 +115,27 @@ trait ShowPartitionsSuiteBase extends QueryTest with SQLTestUtils { } } - test("filter by partitions with all partitioned columns") { + test("filter by partitions") { withNamespace(s"$catalog.ns") { sql(s"CREATE NAMESPACE $catalog.ns") val table = s"$catalog.ns.dateTable" withTable(table) { createDateTable(table) + runShowPartitionsSql( + s"show partitions $table PARTITION(year=2015)", + Row("year=2015/month=1") :: + Row("year=2015/month=2") :: Nil) runShowPartitionsSql( s"show partitions $table PARTITION(year=2015, month=1)", Row("year=2015/month=1") :: Nil) + runShowPartitionsSql( + s"show partitions $table PARTITION(month=2)", + Row("year=2015/month=2") :: + Row("year=2016/month=2") :: Nil) } } } - test("show everything more than 5 part keys") { withNamespace(s"$catalog.ns") { sql(s"CREATE NAMESPACE $catalog.ns") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 33d594d7b214..3600202a2d61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -45,21 +45,6 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { } } - test("filter by partitions") { - val table = "dateTable" - withTable(table) { - createDateTable(table) - runShowPartitionsSql( - s"show partitions default.$table PARTITION(year=2015)", - Row("year=2015/month=1") :: - Row("year=2015/month=2") :: Nil) - runShowPartitionsSql( - s"show partitions default.$table PARTITION(month=2)", - Row("year=2015/month=2") :: - Row("year=2016/month=2") :: Nil) - } - } - // The test fails for V2 Table Catalogs with the exception: // org.apache.spark.sql.AnalysisException: CREATE VIEW is only supported with v1 tables. test("show partitions of a view") { From 1cd1e34b7ba2dc5b2da3ed4acd7674b360646ac5 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 25 Nov 2020 22:01:00 +0300 Subject: [PATCH 36/40] Fix build --- .../sql/execution/datasources/v2/DataSourceV2Strategy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 105dad9b2711..7ef1c28c4e3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -310,7 +310,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case r @ ShowPartitions( ResolvedTable(catalog, _, table: SupportsPartitionManagement), - pattern @ (None | Some(ResolvedPartitionSpec(_, _)))) => + pattern @ (None | Some(_: ResolvedPartitionSpec))) => ShowPartitionsExec( r.output, catalog, From 0fe4a6195617d5a9d5e09ddb9d49c2f273b442d4 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sat, 28 Nov 2020 11:29:11 +0300 Subject: [PATCH 37/40] Fix merge --- .../spark/sql/catalyst/parser/AstBuilder.scala | 5 ++++- .../command/ShowPartitionsParserSuite.scala | 14 +++++++++----- .../execution/command/v1/ShowPartitionsSuite.scala | 8 ++++---- 3 files changed, 17 insertions(+), 10 deletions(-) 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 b13e11340285..017b1df4b9c2 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 @@ -3648,7 +3648,10 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) } ShowPartitions( - UnresolvedTableOrView(multiPart.getOrElse(Seq.empty[String])), + UnresolvedTableOrView( + multiPart.getOrElse(Seq.empty[String]), + "SHOW PARTITIONS", + allowTempView = false), partitionKeys) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala index a382f6c3a716..93541af301ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala @@ -26,17 +26,21 @@ import org.apache.spark.sql.test.SharedSparkSession class ShowPartitionsParserSuite extends AnalysisTest with SharedSparkSession { test("SHOW PARTITIONS") { + val commandName = "SHOW PARTITIONS" Seq( - "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedTableOrView(Seq("t1")), None), - "SHOW PARTITIONS db1.t1" -> ShowPartitions(UnresolvedTableOrView(Seq("db1", "t1")), None), + "SHOW PARTITIONS t1" -> ShowPartitions( + UnresolvedTableOrView(Seq("t1"), commandName, false), None), + "SHOW PARTITIONS db1.t1" -> ShowPartitions( + UnresolvedTableOrView(Seq("db1", "t1"), commandName, false), None), "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" -> ShowPartitions( - UnresolvedTableOrView(Seq("t1")), + UnresolvedTableOrView(Seq("t1"), commandName, false), Some(UnresolvedPartitionSpec(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue")))), - "SHOW PARTITIONS a.b.c" -> ShowPartitions(UnresolvedTableOrView(Seq("a", "b", "c")), None), + "SHOW PARTITIONS a.b.c" -> ShowPartitions( + UnresolvedTableOrView(Seq("a", "b", "c"), commandName, false), None), "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" -> ShowPartitions( - UnresolvedTableOrView(Seq("a", "b", "c")), + UnresolvedTableOrView(Seq("a", "b", "c"), commandName, false), Some(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")))) ).foreach { case (sql, expected) => val parsed = parsePlan(sql) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index 3600202a2d61..d0417f4d960b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -66,10 +66,10 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { val viewName = "test_view" withTempView(viewName) { spark.range(10).createTempView(viewName) - val errMsg = intercept[NoSuchTableException] { + val errMsg = intercept[AnalysisException] { sql(s"SHOW PARTITIONS $viewName") }.getMessage - assert(errMsg.contains(s"Table or view '$viewName' not found")) + assert(errMsg.contains("'SHOW PARTITIONS' expects a table or permanent view")) } } } @@ -84,10 +84,10 @@ class ShowPartitionsSuite extends ShowPartitionsSuiteBase with SharedSparkSessio sql(s""" |CREATE TEMPORARY VIEW $viewName (c1 INT, c2 STRING) |$defaultUsing""".stripMargin) - val errMsg = intercept[NoSuchTableException] { + val errMsg = intercept[AnalysisException] { sql(s"SHOW PARTITIONS $viewName") }.getMessage - assert(errMsg.contains(s"Table or view '$viewName' not found")) + assert(errMsg.contains("'SHOW PARTITIONS' expects a table or permanent view")) } } From d3c344a10a31475e958295592d1ed7b785a5b1af Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Sat, 28 Nov 2020 11:49:54 +0300 Subject: [PATCH 38/40] UnresolvedTableOrView -> UnresolvedTable --- .../spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 4 ---- .../spark/sql/catalyst/parser/AstBuilder.scala | 5 +---- .../command/ShowPartitionsParserSuite.scala | 13 ++++++------- .../execution/command/v1/ShowPartitionsSuite.scala | 7 +++---- 5 files changed, 11 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index dae496244c85..e5482c0ea2d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1084,7 +1084,7 @@ class Analyzer(override val catalogManager: CatalogManager) lookupTableOrView(identifier).map { case v: ResolvedView => val viewStr = if (v.isTemp) "temp view" else "view" - u.failAnalysis(s"${v.identifier.quoted} is a $viewStr. '$cmd' expects a table.'") + u.failAnalysis(s"${v.identifier.quoted} is a $viewStr. '$cmd' expects a table.") case table => table }.getOrElse(u) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 6dfa738fd738..ade0116fd043 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1010,10 +1010,6 @@ trait CheckAnalysis extends PredicateHelper { // Make sure that the `SHOW PARTITIONS` command is allowed for the table private def checkShowPartitions(showPartitions: ShowPartitions): Unit = showPartitions match { - case ShowPartitions(v: ResolvedView, _) => - failAnalysis(s"SHOW PARTITIONS is not allowed on a view: ${v.identifier}") - case ShowPartitions(child, _) if !child.isInstanceOf[ResolvedTable] => - failAnalysis(s"Cannot resolve the table for SHOW PARTITIONS") case ShowPartitions(rt: ResolvedTable, _) if !rt.table.isInstanceOf[SupportsPartitionManagement] => failAnalysis(s"SHOW PARTITIONS cannot run for a table which does not support partitioning") 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 017b1df4b9c2..635ad941f4ca 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 @@ -3648,10 +3648,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) } ShowPartitions( - UnresolvedTableOrView( - multiPart.getOrElse(Seq.empty[String]), - "SHOW PARTITIONS", - allowTempView = false), + UnresolvedTable(multiPart.getOrElse(Seq.empty[String]), "SHOW PARTITIONS"), partitionKeys) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala index 93541af301ab..7b5cf8af4eea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedPartitionSpec, UnresolvedTableOrView} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedPartitionSpec, UnresolvedTable} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.ShowPartitions @@ -28,19 +28,18 @@ class ShowPartitionsParserSuite extends AnalysisTest with SharedSparkSession { test("SHOW PARTITIONS") { val commandName = "SHOW PARTITIONS" Seq( - "SHOW PARTITIONS t1" -> ShowPartitions( - UnresolvedTableOrView(Seq("t1"), commandName, false), None), + "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedTable(Seq("t1"), commandName), None), "SHOW PARTITIONS db1.t1" -> ShowPartitions( - UnresolvedTableOrView(Seq("db1", "t1"), commandName, false), None), + UnresolvedTable(Seq("db1", "t1"), commandName), None), "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')" -> ShowPartitions( - UnresolvedTableOrView(Seq("t1"), commandName, false), + UnresolvedTable(Seq("t1"), commandName), Some(UnresolvedPartitionSpec(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue")))), "SHOW PARTITIONS a.b.c" -> ShowPartitions( - UnresolvedTableOrView(Seq("a", "b", "c"), commandName, false), None), + UnresolvedTable(Seq("a", "b", "c"), commandName), None), "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" -> ShowPartitions( - UnresolvedTableOrView(Seq("a", "b", "c"), commandName, false), + UnresolvedTable(Seq("a", "b", "c"), commandName), Some(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")))) ).foreach { case (sql, expected) => val parsed = parsePlan(sql) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index d0417f4d960b..2b2bc9e63dc8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.command.v1 import org.apache.spark.sql.{AnalysisException, Row, SaveMode} -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.execution.command import org.apache.spark.sql.test.SharedSparkSession @@ -57,7 +56,7 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { val errMsg = intercept[AnalysisException] { sql(s"SHOW PARTITIONS $view") }.getMessage - assert(errMsg.contains("is not allowed on a view")) + assert(errMsg.contains("'SHOW PARTITIONS' expects a table")) } } } @@ -69,7 +68,7 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { val errMsg = intercept[AnalysisException] { sql(s"SHOW PARTITIONS $viewName") }.getMessage - assert(errMsg.contains("'SHOW PARTITIONS' expects a table or permanent view")) + assert(errMsg.contains("'SHOW PARTITIONS' expects a table")) } } } @@ -87,7 +86,7 @@ class ShowPartitionsSuite extends ShowPartitionsSuiteBase with SharedSparkSessio val errMsg = intercept[AnalysisException] { sql(s"SHOW PARTITIONS $viewName") }.getMessage - assert(errMsg.contains("'SHOW PARTITIONS' expects a table or permanent view")) + assert(errMsg.contains("'SHOW PARTITIONS' expects a table")) } } From fbc5820a5775f678cb3311ece59865fcc9385275 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 30 Nov 2020 12:53:53 +0300 Subject: [PATCH 39/40] Address Wenchen's review comment --- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 635ad941f4ca..fd7a649d5dd0 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 @@ -3643,12 +3643,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg * }}} */ override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) { - val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) val partitionKeys = Option(ctx.partitionSpec).map { specCtx => UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) } ShowPartitions( - UnresolvedTable(multiPart.getOrElse(Seq.empty[String]), "SHOW PARTITIONS"), + UnresolvedTable(visitMultipartIdentifier(ctx.multipartIdentifier()), "SHOW PARTITIONS"), partitionKeys) } From dc211e6377a3725d567fd586d835b19659a187fc Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 30 Nov 2020 13:15:40 +0300 Subject: [PATCH 40/40] Remove unused param `tableName` from convertToPartIdent() --- .../spark/sql/catalyst/analysis/ResolvePartitionSpec.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index 4f7a5efa00f1..38991a9e24fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -70,7 +70,6 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { } private def convertToPartIdent( - tableName: String, partitionSpec: TablePartitionSpec, schema: Seq[StructField]): InternalRow = { val partValues = schema.map { part =>