[SPARK-29539][SQL] SHOW PARTITIONS should look up catalog/table like v2 commands

### What changes were proposed in this pull request?
Add ShowPartitionsStatement and make SHOW PARTITIONS go through the same catalog/table resolution framework of v2 commands.

### Why are the changes needed?
It's important to make all the commands have the same table resolution behavior, to avoid confusing end-users.

### Does this PR introduce any user-facing change?
Yes. When running SHOW PARTITIONS, Spark fails the command if the current catalog is set to a v2 catalog, or the table name specified a v2 catalog.

### How was this patch tested?
Unit tests.

Closes #26198 from huaxingao/spark-29539.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
This commit is contained in:
Huaxin Gao 2019-10-22 14:47:17 -07:00 committed by Liang-Chi Hsieh
parent 80094688fd
commit 3bf5355e24
8 changed files with 79 additions and 40 deletions

View file

@ -194,7 +194,7 @@ statement
('(' key=tablePropertyKey ')')? #showTblProperties
| SHOW COLUMNS (FROM | IN) tableIdentifier
((FROM | IN) db=errorCapturingIdentifier)? #showColumns
| SHOW PARTITIONS tableIdentifier partitionSpec? #showPartitions
| SHOW PARTITIONS multipartIdentifier partitionSpec? #showPartitions
| SHOW identifier? FUNCTIONS
(LIKE? (qualifiedName | pattern=STRING))? #showFunctions
| SHOW CREATE TABLE tableIdentifier #showCreateTable

View file

@ -2742,4 +2742,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
visitMultipartIdentifier(ctx.multipartIdentifier),
Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec))
}
/**
* A command for users to list the partition names of a table. If partition spec is specified,
* partitions that match the spec are returned. Otherwise an empty result set is returned.
*
* This function creates a [[ShowPartitionsStatement]] logical plan
*
* The syntax of using this command in SQL is:
* {{{
* SHOW PARTITIONS multi_part_name [partition_spec];
* }}}
*/
override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) {
val table = visitMultipartIdentifier(ctx.multipartIdentifier)
val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)
ShowPartitionsStatement(table, partitionKeys)
}
}

View file

@ -323,3 +323,9 @@ case class RepairTableStatement(tableName: Seq[String]) extends ParsedStatement
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

View file

@ -971,6 +971,32 @@ class DDLParserSuite extends AnalysisTest {
TruncateTableStatement(Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10"))))
}
test("SHOW PARTITIONS") {
val sql1 = "SHOW PARTITIONS t1"
val sql2 = "SHOW PARTITIONS db1.t1"
val sql3 = "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')"
val sql4 = "SHOW PARTITIONS a.b.c"
val sql5 = "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')"
val parsed1 = parsePlan(sql1)
val expected1 = ShowPartitionsStatement(Seq("t1"), None)
val parsed2 = parsePlan(sql2)
val expected2 = ShowPartitionsStatement(Seq("db1", "t1"), None)
val parsed3 = parsePlan(sql3)
val expected3 = ShowPartitionsStatement(Seq("t1"),
Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue")))
val parsed4 = parsePlan(sql4)
val expected4 = ShowPartitionsStatement(Seq("a", "b", "c"), None)
val parsed5 = parsePlan(sql5)
val expected5 = ShowPartitionsStatement(Seq("a", "b", "c"), Some(Map("ds" -> "2017-06-10")))
comparePlans(parsed1, expected1)
comparePlans(parsed2, expected2)
comparePlans(parsed3, expected3)
comparePlans(parsed4, expected4)
comparePlans(parsed5, expected5)
}
private case class TableSpec(
name: Seq[String],
schema: Option[StructType],

View file

@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table}
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowTablesCommand, TruncateTableCommand}
import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand}
import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource}
import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
import org.apache.spark.sql.internal.SQLConf
@ -288,6 +288,12 @@ class ResolveSessionCatalog(
TruncateTableCommand(
v1TableName.asTableIdentifier,
partitionSpec)
case ShowPartitionsStatement(tableName, partitionSpec) =>
val v1TableName = parseV1Table(tableName, "SHOW PARTITIONS")
ShowPartitionsCommand(
v1TableName.asTableIdentifier,
partitionSpec)
}
private def parseV1Table(tableName: Seq[String], sql: String): Seq[String] = {

View file

@ -135,23 +135,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) {
ShowColumnsCommand(Option(ctx.db).map(_.getText), visitTableIdentifier(ctx.tableIdentifier))
}
/**
* A command for users to list the partition names of a table. If partition spec is specified,
* partitions that match the spec are returned. Otherwise an empty result set is returned.
*
* This function creates a [[ShowPartitionsCommand]] logical plan
*
* The syntax of using this command in SQL is:
* {{{
* SHOW PARTITIONS table_identifier [partition_spec];
* }}}
*/
override def visitShowPartitions(ctx: ShowPartitionsContext): LogicalPlan = withOrigin(ctx) {
val table = visitTableIdentifier(ctx.tableIdentifier)
val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)
ShowPartitionsCommand(table, partitionKeys)
}
/**
* Creates a [[ShowCreateTableCommand]]
*/

View file

@ -1232,6 +1232,28 @@ class DataSourceV2SQLSuite
}
}
test("SHOW PARTITIONS") {
val t = "testcat.ns1.ns2.tbl"
withTable(t) {
sql(
s"""
|CREATE TABLE $t (id bigint, data string)
|USING foo
|PARTITIONED BY (id)
""".stripMargin)
val e1 = intercept[AnalysisException] {
val partition = sql(s"SHOW PARTITIONS $t")
}
assert(e1.message.contains("SHOW PARTITIONS is only supported with v1 tables"))
val e2 = intercept[AnalysisException] {
val partition2 = sql(s"SHOW PARTITIONS $t PARTITION(id='1')")
}
assert(e2.message.contains("SHOW PARTITIONS is only supported with v1 tables"))
}
}
private def assertAnalysisError(sqlStatement: String, expectedError: String): Unit = {
val errMsg = intercept[AnalysisException] {
sql(sqlStatement)

View file

@ -870,27 +870,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession {
comparePlans(parsed4, expected4)
}
test("show partitions") {
val sql1 = "SHOW PARTITIONS t1"
val sql2 = "SHOW PARTITIONS db1.t1"
val sql3 = "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', partcol2='partvalue')"
val parsed1 = parser.parsePlan(sql1)
val expected1 =
ShowPartitionsCommand(TableIdentifier("t1", None), None)
val parsed2 = parser.parsePlan(sql2)
val expected2 =
ShowPartitionsCommand(TableIdentifier("t1", Some("db1")), None)
val expected3 =
ShowPartitionsCommand(TableIdentifier("t1", None),
Some(Map("partcol1" -> "partvalue", "partcol2" -> "partvalue")))
val parsed3 = parser.parsePlan(sql3)
comparePlans(parsed1, expected1)
comparePlans(parsed2, expected2)
comparePlans(parsed3, expected3)
}
test("support for other types in DBPROPERTIES") {
val sql =
"""