From 70e1322be70456154600c665b35314f49fcf6a41 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 27 Mar 2026 10:08:37 -0700 Subject: [PATCH 1/8] [SPARK-39660][SQL] Support v2 DESCRIBE TABLE .. PARTITION ## What changes were proposed in this pull request? Enable `DESCRIBE [EXTENDED] TABLE PARTITION (col=val, ...)` for V2 tables that implement `SupportsPartitionManagement`, bringing parity with the V1/Hive behavior. Previously any partition spec on a V2 table threw unconditionally: "DESCRIBE does not support partition for v2 tables." ### Changes: - `DataSourceV2Strategy`: replaced blanket error throw with a capability check (`SupportsPartitionManagement`); passes `partitionSpec` through to `DescribeTableExec`. - `DescribeTableExec`: added `partitionSpec` parameter and new `addPartitionDetails()` method that normalizes the spec, converts it to a typed `InternalRow`, validates partition existence via `loadPartitionMetadata()`, and (for EXTENDED) appends a `# Detailed Partition Information` section. - `QueryCompilationErrors` / `error-conditions.json`: replaced legacy `_LEGACY_ERROR_TEMP_1111` with named error `DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE`. ## How was this patch tested? Replaced the single "not supported" test in `v2/DescribeTableSuite` with four targeted tests: partition validation, extended partition info, non- existent partition error, and capability error for tables without partition management. All 29 V2 + 66 V1 `DescribeTableSuite` tests pass. Co-Authored-By: Claude Sonnet 4.6 --- .../resources/error/error-conditions.json | 6 ++ .../sql/errors/QueryCompilationErrors.scala | 6 +- .../datasources/v2/DataSourceV2Strategy.scala | 6 +- .../datasources/v2/DescribeTableExec.scala | 41 +++++++++++-- .../command/v2/DescribeTableSuite.scala | 60 ++++++++++++++++--- 5 files changed, 101 insertions(+), 18 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index e13e8104dd53..ef5953e33a40 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1938,6 +1938,12 @@ ], "sqlState" : "0A000" }, + "DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE" : { + "message" : [ + "DESCRIBE TABLE ... PARTITION is not supported for table because it does not implement partition management." + ], + "sqlState" : "0A000" + }, "DESCRIBE_JSON_NOT_EXTENDED" : { "message" : [ "DESCRIBE TABLE ... AS JSON only supported when [EXTENDED|FORMATTED] is specified.", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 446c6f2a4784..e900c1fcf4cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1568,10 +1568,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "filters" -> filters.mkString("[", ", ", "]"))) } - def describeDoesNotSupportPartitionForV2TablesError(): Throwable = { + def describePartitionNotSupportedForV2Table(tableName: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1111", - messageParameters = Map.empty) + errorClass = "DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE", + messageParameters = Map("tableName" -> tableName)) } def cannotReplaceMissingTableError( 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 66ce4a637d51..dcc549b1c6db 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 @@ -426,10 +426,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat DescribeNamespaceExec(output, catalog.asNamespaceCatalog, ns, extended) :: Nil case DescribeRelation(r: ResolvedTable, partitionSpec, isExtended, output) => - if (partitionSpec.nonEmpty) { - throw QueryCompilationErrors.describeDoesNotSupportPartitionForV2TablesError() + if (partitionSpec.nonEmpty && !r.table.isInstanceOf[SupportsPartitionManagement]) { + throw QueryCompilationErrors.describePartitionNotSupportedForV2Table(r.name) } - DescribeTableExec(output, r.table, isExtended) :: Nil + DescribeTableExec(output, r.table, isExtended, partitionSpec) :: Nil case DescribeColumn(r: ResolvedTable, column, isExtended, output) => column match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 3cc767380d8b..7a6a48069913 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -21,27 +21,31 @@ import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, ClusterBySpec} +import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec +import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, CatalogTypes, ClusterBySpec} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, ResolveDefaultColumns} -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, SupportsRead, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, SupportsPartitionManagement, SupportsRead, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.{ClusterByTransform, IdentityTransform} import org.apache.spark.sql.connector.read.SupportsReportStatistics import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, PartitioningUtils} import org.apache.spark.util.ArrayImplicits._ case class DescribeTableExec( output: Seq[Attribute], table: Table, - isExtended: Boolean) extends LeafV2CommandExec { + isExtended: Boolean, + partitionSpec: CatalogTypes.TablePartitionSpec = Map.empty) extends LeafV2CommandExec { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() addSchema(rows) addPartitioning(rows) addClustering(rows) - if (isExtended) { + if (partitionSpec.nonEmpty) { + addPartitionDetails(rows) + } else if (isExtended) { addMetadataColumns(rows) addTableDetails(rows) addTableStats(rows) @@ -50,6 +54,33 @@ case class DescribeTableExec( rows.toSeq } + private def addPartitionDetails(rows: ArrayBuffer[InternalRow]): Unit = { + // Guaranteed by DataSourceV2Strategy to be SupportsPartitionManagement + val partTable = table.asInstanceOf[SupportsPartitionManagement] + val partSchema = partTable.partitionSchema() + + // Normalize column names and cast string values to typed InternalRow + val normalizedSpec = PartitioningUtils.normalizePartitionSpec( + partitionSpec, partSchema, table.name(), conf.resolver) + val fields = partSchema.filter(f => normalizedSpec.contains(f.name)) + val partIdent = ResolvePartitionSpec.convertToPartIdent(normalizedSpec, fields) + + // Validates the partition exists - throws NoSuchPartitionException if not found + val metadata = partTable.loadPartitionMetadata(partIdent) + + if (isExtended) { + rows += emptyRow() + rows += toCatalystRow("# Detailed Partition Information", "", "") + rows += toCatalystRow( + "Partition Values", + normalizedSpec.map { case (k, v) => s"$k=$v" }.mkString("[", ", ", "]"), + "") + metadata.asScala.toSeq.sortBy(_._1).foreach { case (k, v) => + rows += toCatalystRow(k, v, "") + } + } + } + private def addTableDetails(rows: ArrayBuffer[InternalRow]): Unit = { rows += emptyRow() rows += toCatalystRow("# Detailed Table Information", "", "") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala index 847a956c11ab..22c51f982306 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala @@ -31,14 +31,60 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase with CommandSuiteBase { import testImplicits._ - test("Describing a partition is not supported") { + test("DESCRIBE TABLE PARTITION validates that the partition exists") { withNamespaceAndTable("ns", "table") { tbl => - spark.sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing " + - "PARTITIONED BY (id)") - val e = intercept[AnalysisException] { - sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") - } - assert(e.message === "DESCRIBE does not support partition for v2 tables.") + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") + // Non-extended describe with a valid partition spec shows schema + partition section, + // same output as DESCRIBE TABLE (partition spec is used only for validation). + checkAnswer( + sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)"), + Seq( + Row("id", "bigint", null), + Row("data", "string", null), + Row("# Partition Information", "", ""), + Row("# col_name", "data_type", "comment"), + Row("id", "bigint", null))) + } + } + + test("DESCRIBE TABLE EXTENDED PARTITION shows detailed partition information") { + withNamespaceAndTable("ns", "table") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") + val result = sql(s"DESCRIBE TABLE EXTENDED $tbl PARTITION (id = 1)").collect() + val colNames = result.map(_.getString(0)) + // Must include the detailed partition section header + assert(colNames.contains("# Detailed Partition Information")) + // Must include the partition values row + assert(result.exists(r => + r.getString(0) == "Partition Values" && r.getString(1) == "[id=1]")) + } + } + + test("DESCRIBE TABLE PARTITION on a non-existent partition raises an error") { + withNamespaceAndTable("ns", "table") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + checkError( + exception = intercept[AnalysisException] { + sql(s"DESCRIBE TABLE $tbl PARTITION (id = 999)").collect() + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "PARTITION (`id` = 999)", + "tableName" -> s"`$catalog`.`ns`.`table`")) + } + } + + test("DESCRIBE TABLE PARTITION on a table without partition management raises an error") { + withNamespaceAndTable("ns", "table", nonPartitionCatalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") + checkError( + exception = intercept[AnalysisException] { + sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") + }, + condition = "DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE", + parameters = Map("tableName" -> tbl)) } } From b089281166f444ccae528e2ab6480051f417f834 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 27 Mar 2026 11:05:56 -0700 Subject: [PATCH 2/8] [SPARK-39660][SQL] Resolve partition spec at analysis time for DESCRIBE TABLE PARTITION Use Option[PartitionSpec] in DescribeRelation and extend V2PartitionCommand so ResolvePartitionSpec handles UnresolvedPartitionSpec -> ResolvedPartitionSpec resolution automatically during analysis, consistent with other partition commands. Move the non-partition-management error check from DataSourceV2Strategy into CheckAnalysis so it is raised as an AnalysisException at analysis time. Co-Authored-By: Claude Sonnet 4.6 --- .../sql/catalyst/analysis/CheckAnalysis.scala | 14 +++++-- .../catalyst/plans/logical/v2Commands.scala | 11 +++--- .../analysis/ResolveSessionCatalog.scala | 6 ++- .../spark/sql/execution/SparkSqlParser.scala | 8 ++-- .../datasources/v2/DataSourceV2Strategy.scala | 6 +-- .../datasources/v2/DescribeTableExec.scala | 37 ++++++++++--------- .../command/DescribeTableParserSuite.scala | 19 +++++++--- .../command/PlanResolutionSuite.scala | 4 +- 8 files changed, 64 insertions(+), 41 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 9d48955cbc71..59d8c671f1d9 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 @@ -370,10 +370,10 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString u.multipartIdentifier, ddlSearchPathForError(catalogPathForError)) - case d: DescribeRelation if !d.relation.resolved => + case d: DescribeRelation if !d.table.resolved => // DESCRIBE TABLE / DESC TABLE: same search path as SELECT // (searchPathForUnresolvedRelation). Relation may be wrapped in SubqueryAlias (e.g. Hive). - firstUnresolvedTableOrView(d.relation).foreach { u => + firstUnresolvedTableOrView(d.table).foreach { u => u.tableNotFound( u.multipartIdentifier, searchPathForUnresolvedRelation(u.multipartIdentifier)) @@ -421,7 +421,15 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString case u: UnresolvedInlineTable if unresolvedInlineTableContainsScalarSubquery(u) => throw QueryCompilationErrors.inlineTableContainsScalarSubquery(u) - case command: V2PartitionCommand => + case d: DescribeRelation if d.partitionSpec.isDefined => + d.table match { + case r @ ResolvedTable(_, _, table, _) + if !table.isInstanceOf[SupportsPartitionManagement] => + throw QueryCompilationErrors.describePartitionNotSupportedForV2Table(r.name) + case _ => + } + + case command: V2PartitionCommand if !command.isInstanceOf[DescribeRelation] => command.table match { case r @ ResolvedTable(_, _, table, _) => table match { case t: SupportsPartitionManagement => 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 b857a360544e..778cd2cb0142 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 @@ -909,13 +909,14 @@ case class SetNamespaceLocation( * The logical plan of the DESCRIBE relation_name command. */ case class DescribeRelation( - relation: LogicalPlan, - partitionSpec: TablePartitionSpec, + table: LogicalPlan, + partitionSpec: Option[PartitionSpec], isExtended: Boolean, - override val output: Seq[Attribute] = DescribeRelation.getOutputAttrs) extends UnaryCommand { - override def child: LogicalPlan = relation + override val output: Seq[Attribute] = DescribeRelation.getOutputAttrs) + extends V2PartitionCommand { + override def allowPartialPartitionSpec: Boolean = false override protected def withNewChildInternal(newChild: LogicalPlan): DescribeRelation = - copy(relation = newChild) + copy(table = newChild) } object DescribeRelation { 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 d7d943a2eedb..a3f72f255877 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 @@ -200,7 +200,11 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) partitionSpec, isExtended, output) => - DescribeTableCommand(resolvedChild, ident, partitionSpec, isExtended, output) + val rawSpec = partitionSpec match { + case Some(UnresolvedPartitionSpec(spec, _)) => spec + case _ => Map.empty[String, String] + } + DescribeTableCommand(resolvedChild, ident, rawSpec, isExtended, output) case DescribeColumn( ResolvedViewIdentifier(ident), column: UnresolvedAttribute, isExtended, output) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index abc62241a006..209235db679c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{CurrentNamespace, GlobalTempView, LocalTempView, PersistedView, PlanWithUnresolvedIdentifier, SchemaEvolution, SchemaTypeEvolution, UnresolvedAttribute, - UnresolvedIdentifier, UnresolvedNamespace, UnresolvedProcedure} + UnresolvedIdentifier, UnresolvedNamespace, UnresolvedPartitionSpec, UnresolvedProcedure} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.catalyst.parser._ @@ -1463,7 +1463,7 @@ class SparkSqlAstBuilder extends AstBuilder { isExtended) } } else { - val partitionSpec = if (ctx.partitionSpec != null) { + val rawSpec = if (ctx.partitionSpec != null) { // According to the syntax, visitPartitionSpec returns `Map[String, Option[String]]`. visitPartitionSpec(ctx.partitionSpec).map { case (key, Some(value)) => key -> value @@ -1474,8 +1474,10 @@ class SparkSqlAstBuilder extends AstBuilder { Map.empty[String, String] } if (asJson) { - DescribeRelationJsonCommand(relation, partitionSpec, isExtended) + // DescribeRelationJsonCommand uses the raw Map directly (V1 path only). + DescribeRelationJsonCommand(relation, rawSpec, isExtended) } else { + val partitionSpec = if (rawSpec.nonEmpty) Some(UnresolvedPartitionSpec(rawSpec)) else None DescribeRelation(relation, partitionSpec, isExtended) } } 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 dcc549b1c6db..950d3e468dce 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 @@ -426,10 +426,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat DescribeNamespaceExec(output, catalog.asNamespaceCatalog, ns, extended) :: Nil case DescribeRelation(r: ResolvedTable, partitionSpec, isExtended, output) => - if (partitionSpec.nonEmpty && !r.table.isInstanceOf[SupportsPartitionManagement]) { - throw QueryCompilationErrors.describePartitionNotSupportedForV2Table(r.name) - } - DescribeTableExec(output, r.table, isExtended, partitionSpec) :: Nil + DescribeTableExec(output, r.table, isExtended, + partitionSpec.map(_.asInstanceOf[ResolvedPartitionSpec])) :: Nil case DescribeColumn(r: ResolvedTable, column, isExtended, output) => column match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 7a6a48069913..6d8a3cea33b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -21,29 +21,30 @@ import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec -import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, CatalogTypes, ClusterBySpec} -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec +import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, ClusterBySpec} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, ResolveDefaultColumns} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, SupportsPartitionManagement, SupportsRead, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.{ClusterByTransform, IdentityTransform} import org.apache.spark.sql.connector.read.SupportsReportStatistics import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.util.{CaseInsensitiveStringMap, PartitioningUtils} +import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ case class DescribeTableExec( output: Seq[Attribute], table: Table, isExtended: Boolean, - partitionSpec: CatalogTypes.TablePartitionSpec = Map.empty) extends LeafV2CommandExec { + partitionSpec: Option[ResolvedPartitionSpec] = None) extends LeafV2CommandExec { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() addSchema(rows) addPartitioning(rows) addClustering(rows) - if (partitionSpec.nonEmpty) { + if (partitionSpec.isDefined) { addPartitionDetails(rows) } else if (isExtended) { addMetadataColumns(rows) @@ -57,24 +58,24 @@ case class DescribeTableExec( private def addPartitionDetails(rows: ArrayBuffer[InternalRow]): Unit = { // Guaranteed by DataSourceV2Strategy to be SupportsPartitionManagement val partTable = table.asInstanceOf[SupportsPartitionManagement] - val partSchema = partTable.partitionSchema() - - // Normalize column names and cast string values to typed InternalRow - val normalizedSpec = PartitioningUtils.normalizePartitionSpec( - partitionSpec, partSchema, table.name(), conf.resolver) - val fields = partSchema.filter(f => normalizedSpec.contains(f.name)) - val partIdent = ResolvePartitionSpec.convertToPartIdent(normalizedSpec, fields) + val spec = partitionSpec.get // guaranteed nonEmpty by caller // Validates the partition exists - throws NoSuchPartitionException if not found - val metadata = partTable.loadPartitionMetadata(partIdent) + val metadata = partTable.loadPartitionMetadata(spec.ident) if (isExtended) { + val partSchema = partTable.partitionSchema() + val timeZoneId = conf.sessionLocalTimeZone + // Render partition values by casting the typed InternalRow back to strings + val partValueStr = spec.names.zipWithIndex.map { case (name, i) => + val dataType = partSchema(partSchema.fieldIndex(name)).dataType + val value = spec.ident.get(i, dataType) + val strVal = Cast(Literal(value, dataType), StringType, Some(timeZoneId)).eval() + s"$name=${if (strVal == null) "null" else strVal.toString}" + }.mkString("[", ", ", "]") rows += emptyRow() rows += toCatalystRow("# Detailed Partition Information", "", "") - rows += toCatalystRow( - "Partition Values", - normalizedSpec.map { case (k, v) => s"$k=$v" }.mkString("[", ", ", "]"), - "") + rows += toCatalystRow("Partition Values", partValueStr, "") metadata.asScala.toSeq.sortBy(_._1).foreach { case (k, v) => rows += toCatalystRow(k, v, "") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala index f8174d24c949..a5c114770d83 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedTableOrView} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, + UnresolvedPartitionSpec, UnresolvedTableOrView} import org.apache.spark.sql.catalyst.plans.logical.{DescribeColumn, DescribeRelation} import org.apache.spark.sql.test.SharedSparkSession @@ -28,16 +29,24 @@ class DescribeTableParserSuite extends SharedSparkSession with AnalysisTest { test("SPARK-17328: Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), Map.empty, isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), None, isExtended = false)) comparePlans(parsePlan("describe table t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), Map.empty, isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), None, isExtended = false)) comparePlans(parsePlan("describe table extended t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), Map.empty, isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), None, isExtended = true)) comparePlans(parsePlan("describe table formatted t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), Map.empty, isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), None, isExtended = true)) + } + + test("describe table with partition spec") { + comparePlans(parsePlan("DESCRIBE TABLE t PARTITION (ds='2024-01-01')"), + DescribeRelation( + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), + Some(UnresolvedPartitionSpec(Map("ds" -> "2024-01-01"))), + isExtended = false)) } test("describe table column") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 92d56e800c72..d4dea931c2e1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -27,7 +27,7 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AnalysisTest, Analyzer, AsOfVersion, EmptyFunctionRegistry, NoSuchTableException, RelationResolution, ResolvedFieldName, ResolvedFieldPosition, ResolvedIdentifier, ResolvedTable, ResolveSessionCatalog, TimeTravelSpec, UnresolvedAttribute, UnresolvedFieldPosition, UnresolvedInlineTable, UnresolvedRelation, UnresolvedSubqueryColumnAliases, UnresolvedTable} +import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AnalysisTest, Analyzer, AsOfVersion, EmptyFunctionRegistry, NoSuchTableException, RelationResolution, ResolvedFieldName, ResolvedFieldPosition, ResolvedIdentifier, ResolvedTable, ResolveSessionCatalog, TimeTravelSpec, UnresolvedAttribute, UnresolvedFieldPosition, UnresolvedInlineTable, UnresolvedPartitionSpec, UnresolvedRelation, UnresolvedSubqueryColumnAliases, UnresolvedTable} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, InMemoryCatalog, SessionCatalog, TempVariableManager} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, Literal, StringLiteral} import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke @@ -967,7 +967,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { parsed3 match { case DescribeRelation(_: ResolvedTable, partitionSpec, isExtended, _) => assert(!isExtended) - assert(partitionSpec == Map("a" -> "1")) + assert(partitionSpec == Some(UnresolvedPartitionSpec(Map("a" -> "1")))) case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) } } From d793ad2a7581ff7a9fb6ebcac68188d8ba93cf51 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 29 Mar 2026 13:46:05 -0700 Subject: [PATCH 3/8] [SPARK-39660][SQL] Fix error condition ordering and update golden file Fix alphabetical ordering of DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE in error-conditions.json (must come after DESCRIBE_JSON_NOT_EXTENDED). Update describe.sql analyzer golden file to reflect new UnresolvedPartitionSpec toString format after DescribeRelation.partitionSpec changed from TablePartitionSpec to Option[PartitionSpec]. Co-Authored-By: Claude Sonnet 4.6 --- .../src/main/resources/error/error-conditions.json | 10 +++++----- .../sql-tests/analyzer-results/describe.sql.out | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index ef5953e33a40..a7e2af84269d 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1938,16 +1938,16 @@ ], "sqlState" : "0A000" }, - "DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE" : { + "DESCRIBE_JSON_NOT_EXTENDED" : { "message" : [ - "DESCRIBE TABLE ... PARTITION is not supported for table because it does not implement partition management." + "DESCRIBE TABLE ... AS JSON only supported when [EXTENDED|FORMATTED] is specified.", + "For example: DESCRIBE EXTENDED AS JSON is supported but DESCRIBE AS JSON is not." ], "sqlState" : "0A000" }, - "DESCRIBE_JSON_NOT_EXTENDED" : { + "DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE" : { "message" : [ - "DESCRIBE TABLE ... AS JSON only supported when [EXTENDED|FORMATTED] is specified.", - "For example: DESCRIBE EXTENDED AS JSON is supported but DESCRIBE AS JSON is not." + "DESCRIBE TABLE ... PARTITION is not supported for table because it does not implement partition management." ], "sqlState" : "0A000" }, diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out index 891d84c0e409..26965263288b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out @@ -379,7 +379,7 @@ ExplainCommand 'DescribeColumn 'b, false, [info_name#x, info_value#x], SimpleMod -- !query EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) -- !query analysis -ExplainCommand 'DescribeRelation [c=Us, d=2], false, [col_name#x, data_type#x, comment#x], SimpleMode +ExplainCommand 'DescribeRelation [unresolvedpartitionspec((c,Us), (d,2), None)], false, [col_name#x, data_type#x, comment#x], SimpleMode -- !query From 3a4c9dad97be6dfd38ec46837d66e09403720f7d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 29 Mar 2026 16:39:21 -0700 Subject: [PATCH 4/8] [SPARK-39660][SQL] Fix describe.sql golden file: Option[PartitionSpec] renders with double brackets Co-Authored-By: Claude Sonnet 4.6 --- .../test/resources/sql-tests/analyzer-results/describe.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out index 26965263288b..5d10134ea058 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out @@ -379,7 +379,7 @@ ExplainCommand 'DescribeColumn 'b, false, [info_name#x, info_value#x], SimpleMod -- !query EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) -- !query analysis -ExplainCommand 'DescribeRelation [unresolvedpartitionspec((c,Us), (d,2), None)], false, [col_name#x, data_type#x, comment#x], SimpleMode +ExplainCommand 'DescribeRelation [[unresolvedpartitionspec((c,Us), (d,2), None)]], false, [col_name#x, data_type#x, comment#x], SimpleMode -- !query From b4f1f6883a8f4202bb60e9b6c4d41c0ce622c2ed Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 29 Mar 2026 19:13:44 -0700 Subject: [PATCH 5/8] [SPARK-39660][SQL] Fix describe.sql golden file: remove extra brackets from DescribeRelation arg Option[PartitionSpec] renders without surrounding brackets in the plan string, unlike the previous Map[String, String] which rendered as [k=v, ...]. Co-Authored-By: Claude Sonnet 4.6 --- .../test/resources/sql-tests/analyzer-results/describe.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out index 5d10134ea058..41df48c3e8ed 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out @@ -379,7 +379,7 @@ ExplainCommand 'DescribeColumn 'b, false, [info_name#x, info_value#x], SimpleMod -- !query EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) -- !query analysis -ExplainCommand 'DescribeRelation [[unresolvedpartitionspec((c,Us), (d,2), None)]], false, [col_name#x, data_type#x, comment#x], SimpleMode +ExplainCommand 'DescribeRelation unresolvedpartitionspec((c,Us), (d,2), None), false, [col_name#x, data_type#x, comment#x], SimpleMode -- !query From 5b9f005b0d28e35f6cbcdf4d7c8cfae89c25c8ac Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 20 Apr 2026 15:53:10 -0700 Subject: [PATCH 6/8] [SPARK-39660][SQL][FOLLOW-UP] Split DescribeTablePartition from DescribeRelation per review Address cloud-fan's review on #55064: split the logical plan into DescribeRelation (pure table describe, no partition) and DescribeTablePartition (extends V2PartitionCommand), mirroring the ShowTables / ShowTablePartition precedent. Key changes: - DescribeRelation reverts to UnaryCommand with no partition spec. - New DescribeTablePartition extends V2PartitionCommand with required partitionSpec, so CheckAnalysis's V2PartitionCommand case naturally covers it without the !isInstanceOf[DescribeRelation] exclusion. - ResolveSessionCatalog has two explicit cases with no silent fallthrough. - New DescribeTablePartitionExec validates partition existence via listPartitionIdentifiers (not loadPartitionMetadata) and renders values with ToPrettyString + escapePathName, consistent with ShowTablePartitionExec. - Remove DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE error class; the scenario is now covered by INVALID_PARTITION_OPERATION. - Add edge-case tests: partial spec, case-insensitive column, unknown column, multi-column partition, NULL value. Co-authored-by: Claude Code --- .../resources/error/error-conditions.json | 6 - .../sql/catalyst/analysis/CheckAnalysis.scala | 14 +- .../catalyst/plans/logical/v2Commands.scala | 25 +++- .../sql/errors/QueryCompilationErrors.scala | 6 - .../connect/pipelines/PipelinesHandler.scala | 3 +- .../analysis/ResolveSessionCatalog.scala | 13 +- .../spark/sql/execution/SparkSqlParser.scala | 5 +- .../datasources/v2/DataSourceV2Strategy.scala | 9 +- .../datasources/v2/DescribeTableExec.scala | 40 +----- .../v2/DescribeTablePartitionExec.scala | 127 ++++++++++++++++++ .../analyzer-results/describe.sql.out | 2 +- .../apache/spark/sql/SQLQueryTestHelper.scala | 1 + .../command/DescribeTableParserSuite.scala | 19 ++- .../command/PlanResolutionSuite.scala | 8 +- .../command/v2/DescribeTableSuite.scala | 77 +++++++++-- .../hive/execution/HiveComparisonTest.scala | 1 + 16 files changed, 260 insertions(+), 96 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTablePartitionExec.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index a7e2af84269d..e13e8104dd53 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1945,12 +1945,6 @@ ], "sqlState" : "0A000" }, - "DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE" : { - "message" : [ - "DESCRIBE TABLE ... PARTITION is not supported for table because it does not implement partition management." - ], - "sqlState" : "0A000" - }, "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED" : { "message" : [ "Distinct window functions are not supported: ." 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 59d8c671f1d9..9d48955cbc71 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 @@ -370,10 +370,10 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString u.multipartIdentifier, ddlSearchPathForError(catalogPathForError)) - case d: DescribeRelation if !d.table.resolved => + case d: DescribeRelation if !d.relation.resolved => // DESCRIBE TABLE / DESC TABLE: same search path as SELECT // (searchPathForUnresolvedRelation). Relation may be wrapped in SubqueryAlias (e.g. Hive). - firstUnresolvedTableOrView(d.table).foreach { u => + firstUnresolvedTableOrView(d.relation).foreach { u => u.tableNotFound( u.multipartIdentifier, searchPathForUnresolvedRelation(u.multipartIdentifier)) @@ -421,15 +421,7 @@ trait CheckAnalysis extends LookupCatalog with QueryErrorsBase with PlanToString case u: UnresolvedInlineTable if unresolvedInlineTableContainsScalarSubquery(u) => throw QueryCompilationErrors.inlineTableContainsScalarSubquery(u) - case d: DescribeRelation if d.partitionSpec.isDefined => - d.table match { - case r @ ResolvedTable(_, _, table, _) - if !table.isInstanceOf[SupportsPartitionManagement] => - throw QueryCompilationErrors.describePartitionNotSupportedForV2Table(r.name) - case _ => - } - - case command: V2PartitionCommand if !command.isInstanceOf[DescribeRelation] => + case command: V2PartitionCommand => command.table match { case r @ ResolvedTable(_, _, table, _) => table match { case t: SupportsPartitionManagement => 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 778cd2cb0142..3850ab8c8ccc 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 @@ -909,20 +909,33 @@ case class SetNamespaceLocation( * The logical plan of the DESCRIBE relation_name command. */ case class DescribeRelation( - table: LogicalPlan, - partitionSpec: Option[PartitionSpec], + relation: LogicalPlan, + partitionSpec: TablePartitionSpec, isExtended: Boolean, - override val output: Seq[Attribute] = DescribeRelation.getOutputAttrs) - extends V2PartitionCommand { - override def allowPartialPartitionSpec: Boolean = false + override val output: Seq[Attribute] = DescribeRelation.getOutputAttrs) extends UnaryCommand { + override def child: LogicalPlan = relation override protected def withNewChildInternal(newChild: LogicalPlan): DescribeRelation = - copy(table = newChild) + copy(relation = newChild) } object DescribeRelation { def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes() } +/** + * The logical plan of the DESCRIBE relation_name PARTITION command. + */ +case class DescribeTablePartition( + table: LogicalPlan, + partitionSpec: PartitionSpec, + isExtended: Boolean, + override val output: Seq[Attribute] = DescribeRelation.getOutputAttrs) + extends V2PartitionCommand { + override def allowPartialPartitionSpec: Boolean = false + override protected def withNewChildInternal(newChild: LogicalPlan): DescribeTablePartition = + copy(table = newChild) +} + /** * The logical plan of the DESCRIBE relation_name col_name command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index e900c1fcf4cc..bd1e876c9fbd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1568,12 +1568,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "filters" -> filters.mkString("[", ", ", "]"))) } - def describePartitionNotSupportedForV2Table(tableName: String): Throwable = { - new AnalysisException( - errorClass = "DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE", - messageParameters = Map("tableName" -> tableName)) - } - def cannotReplaceMissingTableError( tableIdentifier: Identifier): Throwable = { new CannotReplaceMissingTableException(tableIdentifier) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala index 62f060014117..dd4799765040 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/pipelines/PipelinesHandler.scala @@ -28,7 +28,7 @@ import org.apache.spark.connect.proto.{ExecutePlanResponse, PipelineCommandResul import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.plans.logical.{Command, CreateNamespace, CreateTable, CreateTableAsSelect, CreateView, DescribeRelation, DropView, InsertIntoStatement, LogicalPlan, RenameTable, ShowColumns, ShowCreateTable, ShowFunctions, ShowTableProperties, ShowTables, ShowViews} +import org.apache.spark.sql.catalyst.plans.logical.{Command, CreateNamespace, CreateTable, CreateTableAsSelect, CreateView, DescribeRelation, DescribeTablePartition, DropView, InsertIntoStatement, LogicalPlan, RenameTable, ShowColumns, ShowCreateTable, ShowFunctions, ShowTableProperties, ShowTables, ShowViews} import org.apache.spark.sql.connect.common.DataTypeProtoConverter import org.apache.spark.sql.connect.service.SessionHolder import org.apache.spark.sql.execution.command.{ShowCatalogsCommand, ShowNamespacesCommand} @@ -140,6 +140,7 @@ private[connect] object PipelinesHandler extends Logging { def blockUnsupportedSqlCommand(queryPlan: LogicalPlan): Unit = { val allowlistedCommands = Set( classOf[DescribeRelation], + classOf[DescribeTablePartition], classOf[ShowTables], classOf[ShowTableProperties], classOf[ShowNamespacesCommand], 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 a3f72f255877..5c41ba0ac761 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 @@ -200,11 +200,14 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) partitionSpec, isExtended, output) => - val rawSpec = partitionSpec match { - case Some(UnresolvedPartitionSpec(spec, _)) => spec - case _ => Map.empty[String, String] - } - DescribeTableCommand(resolvedChild, ident, rawSpec, isExtended, output) + DescribeTableCommand(resolvedChild, ident, partitionSpec, isExtended, output) + + case DescribeTablePartition( + resolvedChild @ ResolvedV1TableOrViewIdentifier(ident), + UnresolvedPartitionSpec(spec, _), + isExtended, + output) => + DescribeTableCommand(resolvedChild, ident, spec, isExtended, output) case DescribeColumn( ResolvedViewIdentifier(ident), column: UnresolvedAttribute, isExtended, output) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 209235db679c..f4ac3589be64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1476,9 +1476,10 @@ class SparkSqlAstBuilder extends AstBuilder { if (asJson) { // DescribeRelationJsonCommand uses the raw Map directly (V1 path only). DescribeRelationJsonCommand(relation, rawSpec, isExtended) + } else if (rawSpec.nonEmpty) { + DescribeTablePartition(relation, UnresolvedPartitionSpec(rawSpec), isExtended) } else { - val partitionSpec = if (rawSpec.nonEmpty) Some(UnresolvedPartitionSpec(rawSpec)) else None - DescribeRelation(relation, partitionSpec, isExtended) + DescribeRelation(relation, rawSpec, isExtended) } } } 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 950d3e468dce..1d0d41375e9f 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 @@ -425,9 +425,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case DescribeNamespace(ResolvedNamespace(catalog, ns, _), extended, output) => DescribeNamespaceExec(output, catalog.asNamespaceCatalog, ns, extended) :: Nil - case DescribeRelation(r: ResolvedTable, partitionSpec, isExtended, output) => - DescribeTableExec(output, r.table, isExtended, - partitionSpec.map(_.asInstanceOf[ResolvedPartitionSpec])) :: Nil + case DescribeRelation(r: ResolvedTable, _, isExtended, output) => + DescribeTableExec(output, r.table, isExtended) :: Nil + + case DescribeTablePartition(r: ResolvedTable, part, isExtended, output) => + DescribeTablePartitionExec(output, r.table.asPartitionable, r.identifier, + Seq(part).asResolvedPartitionSpecs.head, isExtended) :: Nil case DescribeColumn(r: ResolvedTable, column, isExtended, output) => column match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 6d8a3cea33b7..3cc767380d8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -21,32 +21,27 @@ import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, ClusterBySpec} -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, ResolveDefaultColumns} -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, SupportsPartitionManagement, SupportsRead, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, SupportsRead, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.{ClusterByTransform, IdentityTransform} import org.apache.spark.sql.connector.read.SupportsReportStatistics import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.types.StringType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ case class DescribeTableExec( output: Seq[Attribute], table: Table, - isExtended: Boolean, - partitionSpec: Option[ResolvedPartitionSpec] = None) extends LeafV2CommandExec { + isExtended: Boolean) extends LeafV2CommandExec { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() addSchema(rows) addPartitioning(rows) addClustering(rows) - if (partitionSpec.isDefined) { - addPartitionDetails(rows) - } else if (isExtended) { + if (isExtended) { addMetadataColumns(rows) addTableDetails(rows) addTableStats(rows) @@ -55,33 +50,6 @@ case class DescribeTableExec( rows.toSeq } - private def addPartitionDetails(rows: ArrayBuffer[InternalRow]): Unit = { - // Guaranteed by DataSourceV2Strategy to be SupportsPartitionManagement - val partTable = table.asInstanceOf[SupportsPartitionManagement] - val spec = partitionSpec.get // guaranteed nonEmpty by caller - - // Validates the partition exists - throws NoSuchPartitionException if not found - val metadata = partTable.loadPartitionMetadata(spec.ident) - - if (isExtended) { - val partSchema = partTable.partitionSchema() - val timeZoneId = conf.sessionLocalTimeZone - // Render partition values by casting the typed InternalRow back to strings - val partValueStr = spec.names.zipWithIndex.map { case (name, i) => - val dataType = partSchema(partSchema.fieldIndex(name)).dataType - val value = spec.ident.get(i, dataType) - val strVal = Cast(Literal(value, dataType), StringType, Some(timeZoneId)).eval() - s"$name=${if (strVal == null) "null" else strVal.toString}" - }.mkString("[", ", ", "]") - rows += emptyRow() - rows += toCatalystRow("# Detailed Partition Information", "", "") - rows += toCatalystRow("Partition Values", partValueStr, "") - metadata.asScala.toSeq.sortBy(_._1).foreach { case (k, v) => - rows += toCatalystRow(k, v, "") - } - } - } - private def addTableDetails(rows: ArrayBuffer[InternalRow]): Unit = { rows += emptyRow() rows += toCatalystRow("# Detailed Table Information", "", "") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTablePartitionExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTablePartitionExec.scala new file mode 100644 index 000000000000..fdc9e76f326d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTablePartitionExec.scala @@ -0,0 +1,127 @@ +/* + * 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 scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName +import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal, ToPrettyString} +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsPartitionManagement} +import org.apache.spark.sql.errors.QueryCompilationErrors + +case class DescribeTablePartitionExec( + output: Seq[Attribute], + table: SupportsPartitionManagement, + tableIdent: Identifier, + partSpec: ResolvedPartitionSpec, + isExtended: Boolean) extends LeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + // Always validate partition existence, even for non-extended describe. + validatePartitionExists() + + val rows = new ArrayBuffer[InternalRow]() + addSchema(rows) + addPartitioning(rows) + + if (isExtended) { + addPartitionDetails(rows) + } + rows.toSeq + } + + private def validatePartitionExists(): Unit = { + val partitionSchema = table.partitionSchema() + val (names, ident) = (partSpec.names, partSpec.ident) + val partitionIdentifiers = table.listPartitionIdentifiers(names.toArray, ident) + if (partitionIdentifiers.isEmpty) { + throw QueryCompilationErrors.notExistPartitionError(tableIdent, ident, partitionSchema) + } + } + + private def addPartitionDetails(rows: ArrayBuffer[InternalRow]): Unit = { + val partitionSchema = table.partitionSchema() + val (names, ident) = (partSpec.names, partSpec.ident) + + // Re-list to obtain the canonical partition row for rendering. + val partitionIdentifiers = table.listPartitionIdentifiers(names.toArray, ident) + assert(partitionIdentifiers.length == 1) + val row = partitionIdentifiers.head + + // Render partition values using ToPrettyString + escapePathName, + // consistent with ShowTablePartitionExec. + val len = partitionSchema.length + val partitions = new Array[String](len) + val timeZoneId = conf.sessionLocalTimeZone + for (i <- 0 until len) { + val dataType = partitionSchema(i).dataType + val partValueUTF8String = ToPrettyString(Literal(row.get(i, dataType), dataType), + Some(timeZoneId)).eval(null) + val partValueStr = if (partValueUTF8String == null) "null" else partValueUTF8String.toString + partitions(i) = escapePathName(partitionSchema(i).name) + "=" + escapePathName(partValueStr) + } + val partitionValues = partitions.mkString("[", ", ", "]") + + rows += emptyRow() + rows += toCatalystRow("# Detailed Partition Information", "", "") + rows += toCatalystRow("Partition Values", partitionValues, "") + + val metadata = table.loadPartitionMetadata(ident) + metadata.asScala.toSeq.sortBy(_._1).foreach { case (k, v) => + rows += toCatalystRow(k, v, "") + } + } + + private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { + rows ++= table.columns().map { column => + toCatalystRow(column.name, column.dataType.simpleString, column.comment) + } + } + + private def addPartitioning(rows: ArrayBuffer[InternalRow]): Unit = { + import org.apache.spark.sql.connector.catalog.CatalogV2Util + import org.apache.spark.sql.connector.expressions.IdentityTransform + import org.apache.spark.sql.catalyst.util.quoteIfNeeded + + val partitioning = table.partitioning + if (partitioning.nonEmpty) { + val partitionColumnsOnly = partitioning.forall(t => t.isInstanceOf[IdentityTransform]) + if (partitionColumnsOnly) { + rows += toCatalystRow("# Partition Information", "", "") + rows += toCatalystRow(s"# ${output(0).name}", output(1).name, output(2).name) + val schema = CatalogV2Util.v2ColumnsToStructType(table.columns()) + rows ++= partitioning + .map(_.asInstanceOf[IdentityTransform].ref.fieldNames()) + .map { fieldNames => + val nestedField = schema.findNestedField(fieldNames.toIndexedSeq) + nestedField.get + }.map { case (path, field) => + toCatalystRow( + (path :+ field.name).map(quoteIfNeeded(_)).mkString("."), + field.dataType.simpleString, + field.getComment().orNull) + } + } + } + } + + private def emptyRow(): InternalRow = toCatalystRow("", "", "") +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out index 41df48c3e8ed..f284b4cabdb4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out @@ -379,7 +379,7 @@ ExplainCommand 'DescribeColumn 'b, false, [info_name#x, info_value#x], SimpleMod -- !query EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) -- !query analysis -ExplainCommand 'DescribeRelation unresolvedpartitionspec((c,Us), (d,2), None), false, [col_name#x, data_type#x, comment#x], SimpleMode +ExplainCommand 'DescribeTablePartition unresolvedpartitionspec((c,Us), (d,2), None), false, [col_name#x, data_type#x, comment#x], SimpleMode -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala index 5ad9605c7d78..3cf26aa94a5d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -137,6 +137,7 @@ trait SQLQueryTestHelper extends SQLConfHelper with Logging { case _: DescribeCommandBase | _: DescribeColumnCommand | _: DescribeRelation + | _: DescribeTablePartition | _: DescribeColumn => true case PhysicalOperation(_, _, Sort(_, true, _, _)) => true case _ => plan.children.iterator.exists(isSemanticallySorted) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala index a5c114770d83..0211bfd0de68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedPartitionSpec, UnresolvedTableOrView} -import org.apache.spark.sql.catalyst.plans.logical.{DescribeColumn, DescribeRelation} +import org.apache.spark.sql.catalyst.plans.logical.{DescribeColumn, DescribeRelation, + DescribeTablePartition} import org.apache.spark.sql.test.SharedSparkSession class DescribeTableParserSuite extends SharedSparkSession with AnalysisTest { @@ -29,23 +30,27 @@ class DescribeTableParserSuite extends SharedSparkSession with AnalysisTest { test("SPARK-17328: Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), None, isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), + Map.empty, isExtended = false)) comparePlans(parsePlan("describe table t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), None, isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), + Map.empty, isExtended = false)) comparePlans(parsePlan("describe table extended t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), None, isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), + Map.empty, isExtended = true)) comparePlans(parsePlan("describe table formatted t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), None, isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), + Map.empty, isExtended = true)) } test("describe table with partition spec") { comparePlans(parsePlan("DESCRIBE TABLE t PARTITION (ds='2024-01-01')"), - DescribeRelation( + DescribeTablePartition( UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), - Some(UnresolvedPartitionSpec(Map("ds" -> "2024-01-01"))), + UnresolvedPartitionSpec(Map("ds" -> "2024-01-01")), isExtended = false)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index d4dea931c2e1..57fbfa69b471 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, EqualTo, Expression, InSubquery, IntegerLiteral, ListQuery, Literal, StringLiteral} import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.{AlterColumns, AlterColumnSpec, AnalysisOnlyCommand, AppendData, Assignment, CreateTable, CreateTableAsSelect, DefaultValueExpression, DeleteAction, DeleteFromTable, DescribeRelation, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, OverwriteByExpression, OverwritePartitionsDynamic, Project, SetTableLocation, SetTableProperties, ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, UpdateTable} +import org.apache.spark.sql.catalyst.plans.logical.{AlterColumns, AlterColumnSpec, AnalysisOnlyCommand, AppendData, Assignment, CreateTable, CreateTableAsSelect, DefaultValueExpression, DeleteAction, DeleteFromTable, DescribeRelation, DescribeTablePartition, DropTable, InsertAction, InsertIntoStatement, LocalRelation, LogicalPlan, MergeIntoTable, OneRowRelation, OverwriteByExpression, OverwritePartitionsDynamic, Project, SetTableLocation, SetTableProperties, ShowTableProperties, SubqueryAlias, UnsetTableProperties, UpdateAction, UpdateTable} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.connector.FakeV2Provider @@ -965,10 +965,10 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { comparePlans(parsed3, expected3) } else { parsed3 match { - case DescribeRelation(_: ResolvedTable, partitionSpec, isExtended, _) => + case DescribeTablePartition(_: ResolvedTable, partitionSpec, isExtended, _) => assert(!isExtended) - assert(partitionSpec == Some(UnresolvedPartitionSpec(Map("a" -> "1")))) - case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) + assert(partitionSpec == UnresolvedPartitionSpec(Map("a" -> "1"))) + case _ => fail("Expect DescribeTablePartition, but got:\n" + parsed3.treeString) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala index 22c51f982306..93b78c362894 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala @@ -31,7 +31,7 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase with CommandSuiteBase { import testImplicits._ - test("DESCRIBE TABLE PARTITION validates that the partition exists") { + test("DESCRIBE TABLE PARTITION on an existing partition") { withNamespaceAndTable("ns", "table") { tbl => sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") @@ -72,19 +72,80 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase condition = "PARTITIONS_NOT_FOUND", parameters = Map( "partitionList" -> "PARTITION (`id` = 999)", - "tableName" -> s"`$catalog`.`ns`.`table`")) + "tableName" -> "`ns`.`table`")) } } test("DESCRIBE TABLE PARTITION on a table without partition management raises an error") { withNamespaceAndTable("ns", "table", nonPartitionCatalog) { tbl => sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") - checkError( - exception = intercept[AnalysisException] { - sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") - }, - condition = "DESCRIBE_PARTITION_NOT_SUPPORTED_FOR_V2_TABLE", - parameters = Map("tableName" -> tbl)) + val e = intercept[AnalysisException] { + sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") + } + assert(e.getCondition == + "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED" || + e.getCondition == "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY") + } + } + + test("DESCRIBE TABLE PARTITION with partial partition spec raises an error") { + withNamespaceAndTable("ns", "table") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, city string, data string) " + + s"$defaultUsing PARTITIONED BY (id, city)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1, city = 'NYC')") + val e = intercept[AnalysisException] { + sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") + } + assert(e.getMessage.contains("id") && e.getMessage.contains("city")) + } + } + + test("DESCRIBE TABLE PARTITION with case-insensitive partition column name") { + withNamespaceAndTable("ns", "table") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") + // Case-insensitive partition column name should normalize. + checkAnswer( + sql(s"DESCRIBE TABLE $tbl PARTITION (ID = 1)"), + Seq( + Row("id", "bigint", null), + Row("data", "string", null), + Row("# Partition Information", "", ""), + Row("# col_name", "data_type", "comment"), + Row("id", "bigint", null))) + } + } + + test("DESCRIBE TABLE PARTITION with unknown partition column raises an error") { + withNamespaceAndTable("ns", "table") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + val e = intercept[AnalysisException] { + sql(s"DESCRIBE TABLE $tbl PARTITION (xyz = 1)") + } + assert(e.getMessage.contains("xyz")) + } + } + + test("DESCRIBE TABLE EXTENDED PARTITION with multi-column partition") { + withNamespaceAndTable("ns", "table") { tbl => + sql(s"CREATE TABLE $tbl (data string, id bigint, city string) " + + s"$defaultUsing PARTITIONED BY (id, city)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1, city = 'NYC')") + val result = sql(s"DESCRIBE TABLE EXTENDED $tbl PARTITION (id = 1, city = 'NYC')").collect() + assert(result.exists(r => + r.getString(0) == "Partition Values" && r.getString(1) == "[id=1, city=NYC]")) + } + } + + test("DESCRIBE TABLE EXTENDED PARTITION with NULL partition value") { + withNamespaceAndTable("ns", "table") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = null)") + val result = sql(s"DESCRIBE TABLE EXTENDED $tbl PARTITION (id = null)").collect() + val partRow = result.find(_.getString(0) == "Partition Values") + assert(partRow.isDefined, s"Partition Values row not found in: ${result.mkString("\n")}") + assert(partRow.get.getString(1) == "[id=NULL]" || partRow.get.getString(1) == "[id=null]", + s"Unexpected partition value: ${partRow.get.getString(1)}") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 1f52e25a730b..bbb7d9ea5b76 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -375,6 +375,7 @@ abstract class HiveComparisonTest extends SparkFunSuite { (!hiveQuery.logical.isInstanceOf[DescribeFunction]) && (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && (!hiveQuery.logical.isInstanceOf[DescribeRelation]) && + (!hiveQuery.logical.isInstanceOf[DescribeTablePartition]) && (!hiveQuery.logical.isInstanceOf[DescribeColumn]) && preparedHive != catalyst) { From 42a99a746636c7546c9096cf0a5e6a4ada2b943d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 21 Apr 2026 00:30:26 -0700 Subject: [PATCH 7/8] [SPARK-39660][SQL][FOLLOW-UP] Address second-round review comments - Delegate base rows (schema + partitioning + clustering) from DescribeTablePartitionExec to DescribeTableExec.addBaseDescription, eliminating duplicated code and three divergences (missing ClusterByTransform filter, cryptic nestedField.get error, missing non-identity else branch) plus the missing addClustering call. - Cache listPartitionIdentifiers result to avoid double round-trip. - Drop partitionSpec field from DescribeRelation (always empty). - Drop redundant allowPartialPartitionSpec override in DescribeTablePartition (default is already false). - Delete orphaned _LEGACY_ERROR_TEMP_1111 error class. - Tighten test assertions: use checkError with exact conditions (_LEGACY_ERROR_TEMP_1232, PARTITIONS_NOT_FOUND, INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED) instead of loose getMessage.contains checks. - Fix NULL partition value assertion to expect "[id=NULL]" only. Co-authored-by: Claude Code --- .../resources/error/error-conditions.json | 5 -- .../catalyst/plans/logical/v2Commands.scala | 2 - .../analysis/ResolveSessionCatalog.scala | 3 +- .../spark/sql/execution/SparkSqlParser.scala | 2 +- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../datasources/v2/DescribeTableExec.scala | 11 +++- .../v2/DescribeTablePartitionExec.scala | 64 ++++--------------- .../command/DescribeTableParserSuite.scala | 12 ++-- .../command/PlanResolutionSuite.scala | 6 +- .../command/v2/DescribeTableSuite.scala | 44 ++++++++----- 10 files changed, 60 insertions(+), 91 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index e13e8104dd53..e6c786640e09 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -8979,11 +8979,6 @@ "Cannot delete from table where ." ] }, - "_LEGACY_ERROR_TEMP_1111" : { - "message" : [ - "DESCRIBE does not support partition for v2 tables." - ] - }, "_LEGACY_ERROR_TEMP_1114" : { "message" : [ "The streaming sources in a query do not have a common supported execution mode.", 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 3850ab8c8ccc..f63de8d1e465 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 @@ -910,7 +910,6 @@ case class SetNamespaceLocation( */ case class DescribeRelation( relation: LogicalPlan, - partitionSpec: TablePartitionSpec, isExtended: Boolean, override val output: Seq[Attribute] = DescribeRelation.getOutputAttrs) extends UnaryCommand { override def child: LogicalPlan = relation @@ -931,7 +930,6 @@ case class DescribeTablePartition( isExtended: Boolean, override val output: Seq[Attribute] = DescribeRelation.getOutputAttrs) extends V2PartitionCommand { - override def allowPartialPartitionSpec: Boolean = false override protected def withNewChildInternal(newChild: LogicalPlan): DescribeTablePartition = copy(table = newChild) } 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 5c41ba0ac761..e2bfaef1e700 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 @@ -197,10 +197,9 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. case DescribeRelation( resolvedChild @ ResolvedV1TableOrViewIdentifier(ident), - partitionSpec, isExtended, output) => - DescribeTableCommand(resolvedChild, ident, partitionSpec, isExtended, output) + DescribeTableCommand(resolvedChild, ident, Map.empty, isExtended, output) case DescribeTablePartition( resolvedChild @ ResolvedV1TableOrViewIdentifier(ident), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index f4ac3589be64..8f4f7b0dd23f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1479,7 +1479,7 @@ class SparkSqlAstBuilder extends AstBuilder { } else if (rawSpec.nonEmpty) { DescribeTablePartition(relation, UnresolvedPartitionSpec(rawSpec), isExtended) } else { - DescribeRelation(relation, rawSpec, isExtended) + DescribeRelation(relation, isExtended) } } } 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 1d0d41375e9f..6730673cab02 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 @@ -425,7 +425,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case DescribeNamespace(ResolvedNamespace(catalog, ns, _), extended, output) => DescribeNamespaceExec(output, catalog.asNamespaceCatalog, ns, extended) :: Nil - case DescribeRelation(r: ResolvedTable, _, isExtended, output) => + case DescribeRelation(r: ResolvedTable, isExtended, output) => DescribeTableExec(output, r.table, isExtended) :: Nil case DescribeTablePartition(r: ResolvedTable, part, isExtended, output) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 3cc767380d8b..51f906724b98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -37,9 +37,7 @@ case class DescribeTableExec( isExtended: Boolean) extends LeafV2CommandExec { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - addSchema(rows) - addPartitioning(rows) - addClustering(rows) + addBaseDescription(rows) if (isExtended) { addMetadataColumns(rows) @@ -50,6 +48,13 @@ case class DescribeTableExec( rows.toSeq } + /** Schema + partitioning + clustering rows, shared with DescribeTablePartitionExec. */ + private[v2] def addBaseDescription(rows: ArrayBuffer[InternalRow]): Unit = { + addSchema(rows) + addPartitioning(rows) + addClustering(rows) + } + private def addTableDetails(rows: ArrayBuffer[InternalRow]): Unit = { rows += emptyRow() rows += toCatalystRow("# Detailed Table Information", "", "") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTablePartitionExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTablePartitionExec.scala index fdc9e76f326d..6a2c3c441f8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTablePartitionExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTablePartitionExec.scala @@ -35,36 +35,33 @@ case class DescribeTablePartitionExec( isExtended: Boolean) extends LeafV2CommandExec { override protected def run(): Seq[InternalRow] = { - // Always validate partition existence, even for non-extended describe. - validatePartitionExists() + val partitionRow = validateAndGetPartition() + // Delegate schema + partitioning + clustering to DescribeTableExec. val rows = new ArrayBuffer[InternalRow]() - addSchema(rows) - addPartitioning(rows) + DescribeTableExec(output, table, isExtended = false).addBaseDescription(rows) if (isExtended) { - addPartitionDetails(rows) + addPartitionDetails(rows, partitionRow) } rows.toSeq } - private def validatePartitionExists(): Unit = { + private def validateAndGetPartition(): InternalRow = { val partitionSchema = table.partitionSchema() val (names, ident) = (partSpec.names, partSpec.ident) val partitionIdentifiers = table.listPartitionIdentifiers(names.toArray, ident) if (partitionIdentifiers.isEmpty) { throw QueryCompilationErrors.notExistPartitionError(tableIdent, ident, partitionSchema) } + assert(partitionIdentifiers.length == 1) + partitionIdentifiers.head } - private def addPartitionDetails(rows: ArrayBuffer[InternalRow]): Unit = { + private def addPartitionDetails( + rows: ArrayBuffer[InternalRow], + partitionRow: InternalRow): Unit = { val partitionSchema = table.partitionSchema() - val (names, ident) = (partSpec.names, partSpec.ident) - - // Re-list to obtain the canonical partition row for rendering. - val partitionIdentifiers = table.listPartitionIdentifiers(names.toArray, ident) - assert(partitionIdentifiers.length == 1) - val row = partitionIdentifiers.head // Render partition values using ToPrettyString + escapePathName, // consistent with ShowTablePartitionExec. @@ -73,55 +70,20 @@ case class DescribeTablePartitionExec( val timeZoneId = conf.sessionLocalTimeZone for (i <- 0 until len) { val dataType = partitionSchema(i).dataType - val partValueUTF8String = ToPrettyString(Literal(row.get(i, dataType), dataType), + val partValueUTF8String = ToPrettyString(Literal(partitionRow.get(i, dataType), dataType), Some(timeZoneId)).eval(null) val partValueStr = if (partValueUTF8String == null) "null" else partValueUTF8String.toString partitions(i) = escapePathName(partitionSchema(i).name) + "=" + escapePathName(partValueStr) } val partitionValues = partitions.mkString("[", ", ", "]") - rows += emptyRow() + rows += toCatalystRow("", "", "") rows += toCatalystRow("# Detailed Partition Information", "", "") rows += toCatalystRow("Partition Values", partitionValues, "") - val metadata = table.loadPartitionMetadata(ident) + val metadata = table.loadPartitionMetadata(partSpec.ident) metadata.asScala.toSeq.sortBy(_._1).foreach { case (k, v) => rows += toCatalystRow(k, v, "") } } - - private def addSchema(rows: ArrayBuffer[InternalRow]): Unit = { - rows ++= table.columns().map { column => - toCatalystRow(column.name, column.dataType.simpleString, column.comment) - } - } - - private def addPartitioning(rows: ArrayBuffer[InternalRow]): Unit = { - import org.apache.spark.sql.connector.catalog.CatalogV2Util - import org.apache.spark.sql.connector.expressions.IdentityTransform - import org.apache.spark.sql.catalyst.util.quoteIfNeeded - - val partitioning = table.partitioning - if (partitioning.nonEmpty) { - val partitionColumnsOnly = partitioning.forall(t => t.isInstanceOf[IdentityTransform]) - if (partitionColumnsOnly) { - rows += toCatalystRow("# Partition Information", "", "") - rows += toCatalystRow(s"# ${output(0).name}", output(1).name, output(2).name) - val schema = CatalogV2Util.v2ColumnsToStructType(table.columns()) - rows ++= partitioning - .map(_.asInstanceOf[IdentityTransform].ref.fieldNames()) - .map { fieldNames => - val nestedField = schema.findNestedField(fieldNames.toIndexedSeq) - nestedField.get - }.map { case (path, field) => - toCatalystRow( - (path :+ field.name).map(quoteIfNeeded(_)).mkString("."), - field.dataType.simpleString, - field.getComment().orNull) - } - } - } - } - - private def emptyRow(): InternalRow = toCatalystRow("", "", "") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala index 0211bfd0de68..436fa2e2389a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableParserSuite.scala @@ -30,20 +30,16 @@ class DescribeTableParserSuite extends SharedSparkSession with AnalysisTest { test("SPARK-17328: Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), - Map.empty, isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), isExtended = false)) comparePlans(parsePlan("describe table t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), - Map.empty, isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), isExtended = false)) comparePlans(parsePlan("describe table extended t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), - Map.empty, isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), isExtended = true)) comparePlans(parsePlan("describe table formatted t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), - Map.empty, isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), isExtended = true)) } test("describe table with partition spec") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 57fbfa69b471..89fb6eca223e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -943,13 +943,13 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { comparePlans(parsed2, expected2) } else { parsed1 match { - case DescribeRelation(_: ResolvedTable, _, isExtended, _) => + case DescribeRelation(_: ResolvedTable, isExtended, _) => assert(!isExtended) case _ => fail("Expect DescribeTable, but got:\n" + parsed1.treeString) } parsed2 match { - case DescribeRelation(_: ResolvedTable, _, isExtended, _) => + case DescribeRelation(_: ResolvedTable, isExtended, _) => assert(isExtended) case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) } @@ -1662,7 +1662,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { case AppendData(r: DataSourceV2Relation, _, _, _, _, _, _) => assert(r.catalog.contains(catalog)) assert(r.identifier.exists(_.name() == tableIdent)) - case DescribeRelation(r: ResolvedTable, _, _, _) => + case DescribeRelation(r: ResolvedTable, _, _) => assert(r.catalog == catalog) assert(r.identifier.name() == tableIdent) case ShowTableProperties(r: ResolvedTable, _, _) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala index 93b78c362894..1bd6df5cf928 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/DescribeTableSuite.scala @@ -79,12 +79,17 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase test("DESCRIBE TABLE PARTITION on a table without partition management raises an error") { withNamespaceAndTable("ns", "table", nonPartitionCatalog) { tbl => sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") - val e = intercept[AnalysisException] { - sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") - } - assert(e.getCondition == - "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED" || - e.getCondition == "INVALID_PARTITION_OPERATION.PARTITION_SCHEMA_IS_EMPTY") + val sqlText = s"DESCRIBE TABLE $tbl PARTITION (id = 1)" + checkError( + exception = intercept[AnalysisException] { + sql(sqlText) + }, + condition = "INVALID_PARTITION_OPERATION.PARTITION_MANAGEMENT_IS_UNSUPPORTED", + parameters = Map("name" -> s"`$nonPartitionCatalog`.`ns`.`table`"), + queryContext = Array(ExpectedContext( + fragment = tbl, + start = sqlText.indexOf(tbl), + stop = sqlText.indexOf(tbl) + tbl.length - 1))) } } @@ -93,10 +98,15 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase sql(s"CREATE TABLE $tbl (id bigint, city string, data string) " + s"$defaultUsing PARTITIONED BY (id, city)") sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1, city = 'NYC')") - val e = intercept[AnalysisException] { - sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") - } - assert(e.getMessage.contains("id") && e.getMessage.contains("city")) + checkError( + exception = intercept[AnalysisException] { + sql(s"DESCRIBE TABLE $tbl PARTITION (id = 1)") + }, + condition = "_LEGACY_ERROR_TEMP_1232", + parameters = Map( + "specKeys" -> "id", + "partitionColumnNames" -> "id, city", + "tableName" -> s"`$catalog`.`ns`.`table`")) } } @@ -119,10 +129,14 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase test("DESCRIBE TABLE PARTITION with unknown partition column raises an error") { withNamespaceAndTable("ns", "table") { tbl => sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") - val e = intercept[AnalysisException] { - sql(s"DESCRIBE TABLE $tbl PARTITION (xyz = 1)") - } - assert(e.getMessage.contains("xyz")) + checkError( + exception = intercept[AnalysisException] { + sql(s"DESCRIBE TABLE $tbl PARTITION (xyz = 1)") + }, + condition = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "`xyz`", + "tableName" -> s"`$catalog`.`ns`.`table`")) } } @@ -144,7 +158,7 @@ class DescribeTableSuite extends command.DescribeTableSuiteBase val result = sql(s"DESCRIBE TABLE EXTENDED $tbl PARTITION (id = null)").collect() val partRow = result.find(_.getString(0) == "Partition Values") assert(partRow.isDefined, s"Partition Values row not found in: ${result.mkString("\n")}") - assert(partRow.get.getString(1) == "[id=NULL]" || partRow.get.getString(1) == "[id=null]", + assert(partRow.get.getString(1) == "[id=NULL]", s"Unexpected partition value: ${partRow.get.getString(1)}") } } From 77a39c115193717ce2454d7ac2dd6f8a40a598db Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 22 Apr 2026 07:16:57 -0700 Subject: [PATCH 8/8] [SPARK-39660][SQL][FOLLOW-UP] Fix stale Scaladoc in visitDescribeRelation Fix pre-existing typo (DescribeRelationAsJsonCommand -> DescribeRelationJsonCommand) and add DescribeTablePartition to the Scaladoc. Co-authored-by: Claude Code --- .../scala/org/apache/spark/sql/execution/SparkSqlParser.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8f4f7b0dd23f..b910a3dd6d8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1440,8 +1440,8 @@ class SparkSqlAstBuilder extends AstBuilder { } /** - * Create a [[DescribeColumn]] or [[DescribeRelation]] or [[DescribeRelationAsJsonCommand]] - * command. + * Create a [[DescribeColumn]], [[DescribeRelation]], [[DescribeTablePartition]], or + * [[DescribeRelationJsonCommand]] command. */ override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) { val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null