From 39f3c42cce2437f3181120179a78b242c3d392f3 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sat, 20 Aug 2022 19:35:27 +0800 Subject: [PATCH 01/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in DSv2 --- .../sql/catalyst/analysis/CheckAnalysis.scala | 4 - .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../catalyst/plans/logical/v2Commands.scala | 2 +- .../sql/errors/QueryCompilationErrors.scala | 4 - .../sql/errors/QueryExecutionErrors.scala | 19 +- .../v2/DataSourceV2Implicits.scala | 5 + .../analysis/ResolveSessionCatalog.scala | 5 +- .../datasources/v2/DataSourceV2Strategy.scala | 13 ++ .../datasources/v2/ShowTablesExec.scala | 162 ++++++++++++++++-- .../command/ShowTablesParserSuite.scala | 8 +- .../command/v2/ShowTablesSuite.scala | 138 +++++++++++---- 11 files changed, 293 insertions(+), 71 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 ec9d8aab3af7..99529d620415 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 @@ -176,10 +176,6 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog { case _ => } - // `ShowTableExtended` should have been converted to the v1 command if the table is v1. - case _: ShowTableExtended => - throw QueryCompilationErrors.commandUnsupportedInV2TableError("SHOW TABLE EXTENDED") - case operator: LogicalPlan => operator transformExpressionsDown { // Check argument data types of higher-order functions downwards first. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 281146d3a38f..2ebb69ebabb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3715,9 +3715,7 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit override def visitShowTableExtended( ctx: ShowTableExtendedContext): LogicalPlan = withOrigin(ctx) { val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) - val partitionKeys = Option(ctx.partitionSpec).map { specCtx => - UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) - } + val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) ShowTableExtended( UnresolvedNamespace(multiPart.getOrElse(Seq.empty[String])), string(ctx.pattern), 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 e5bc96eab8a3..2affd3fa851b 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 @@ -706,7 +706,7 @@ object ShowTables { case class ShowTableExtended( namespace: LogicalPlan, pattern: String, - partitionSpec: Option[PartitionSpec], + partitionSpec: Option[TablePartitionSpec], override val output: Seq[Attribute] = ShowTableExtended.getOutputAttrs) extends UnaryCommand { override def child: LogicalPlan = namespace override protected def withNewChildInternal(newChild: LogicalPlan): ShowTableExtended = 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 8b9663f17347..c9140351bf0e 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 @@ -1559,10 +1559,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase { "definitions will take precedence. See more details in SPARK-28228.") } - def commandUnsupportedInV2TableError(name: String): Throwable = { - new AnalysisException(s"$name is not supported for v2 tables.") - } - def cannotResolveColumnNameAmongAttributesError( colName: String, fieldNames: String): Throwable = { new AnalysisException(s"""Cannot resolve column name "$colName" among ($fieldNames)""") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index e4481a4c7835..2f8ba1b18efd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -26,19 +26,17 @@ import java.time.{DateTimeException, LocalDate} import java.time.temporal.ChronoField import java.util.ConcurrentModificationException import java.util.concurrent.TimeoutException - import com.fasterxml.jackson.core.{JsonParser, JsonToken} import org.apache.hadoop.fs.{FileAlreadyExistsException, FileStatus, Path} import org.apache.hadoop.fs.permission.FsPermission import org.codehaus.commons.compiler.{CompileException, InternalCompilerException} - import org.apache.spark._ import org.apache.spark.executor.CommitDeniedException import org.apache.spark.launcher.SparkLauncher import org.apache.spark.memory.SparkOutOfMemoryError -import org.apache.spark.sql.catalyst.{TableIdentifier, WalkedTypePath} +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier, WalkedTypePath} import org.apache.spark.sql.catalyst.ScalaReflection.Schema -import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator +import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, UnresolvedGenerator} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.parser.ParseException @@ -2089,4 +2087,17 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { toSQLId(funcName), pattern)) } + + def showTableExtendedMultiPartitionUnsupportedError(tableName: String): Throwable = { + new UnsupportedOperationException( + s"The table $tableName does not support show table extended of multiple partition.") + } + + def notExistPartitionError( + tableName: String, + partitionIdent: InternalRow, + partitionSchema: StructType): Throwable = { + new NoSuchPartitionException(tableName, partitionIdent, partitionSchema) + } + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index 1bafcffc3259..713d210c1606 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -65,6 +65,11 @@ object DataSourceV2Implicits { } } + def isPartitionable: Boolean = table match { + case _: SupportsPartitionManagement => true + case _ => false + } + def asPartitionable: SupportsPartitionManagement = { table match { case support: SupportsPartitionManagement => 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 e168987189d3..ee01cf41980f 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 @@ -234,7 +234,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case ShowTableExtended( DatabaseInSessionCatalog(db), pattern, - partitionSpec @ (None | Some(UnresolvedPartitionSpec(_, _))), + partitionSpec, output) => val newOutput = if (conf.getConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA)) { assert(output.length == 4) @@ -242,8 +242,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } else { output } - val tablePartitionSpec = partitionSpec.map(_.asInstanceOf[UnresolvedPartitionSpec].spec) - ShowTablesCommand(Some(db), Some(pattern), newOutput, true, tablePartitionSpec) + ShowTablesCommand(Some(db), Some(pattern), newOutput, true, partitionSpec) // ANALYZE TABLE works on permanent views if the views are cached. case AnalyzeTable(ResolvedV1TableOrViewIdentifier(ident), partitionSpec, noScan) => 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 4bbbac98ca31..675cdff18b9f 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 @@ -368,6 +368,19 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case ShowTables(ResolvedNamespace(catalog, ns), pattern, output) => ShowTablesExec(output, catalog.asTableCatalog, ns, pattern) :: Nil + case ShowTableExtended( + ResolvedNamespace(catalog, ns), + pattern, + partitionSpec, + output) => + ShowTablesExec( + output, + catalog.asTableCatalog, + ns, + Some(pattern), + true, + partitionSpec) :: Nil + case SetCatalogAndNamespace(ResolvedNamespace(catalog, ns)) => val catalogManager = session.sessionState.catalogManager val namespace = if (ns.nonEmpty) Some(ns) else None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index cde3dfb7ead5..4e6d12c16d7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -17,14 +17,24 @@ package org.apache.spark.sql.execution.datasources.v2 +import scala.collection.JavaConverters._ +import scala.collection.convert.ImplicitConversions.`map AsScala` +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer - import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.util.StringUtils -import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} -import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper +import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec.convertToPartIdent +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} +import org.apache.spark.sql.catalyst.util.{StringUtils, quoteIdentifier} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, SupportsPartitionManagement, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper +import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.util.PartitioningUtils.{normalizePartitionSpec, requireExactMatchedPartitionSpec} /** * Physical plan node for showing tables. @@ -33,14 +43,37 @@ case class ShowTablesExec( output: Seq[Attribute], catalog: TableCatalog, namespace: Seq[String], - pattern: Option[String]) extends V2CommandExec with LeafExecNode { + pattern: Option[String], + isExtended: Boolean = false, + partitionSpec: Option[TablePartitionSpec] = None) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - val tables = catalog.listTables(namespace.toArray) - tables.map { table => - if (pattern.map(StringUtils.filterPattern(Seq(table.name()), _).nonEmpty).getOrElse(true)) { - rows += toCatalystRow(table.namespace().quoted, table.name(), isTempView(table)) + val identifiers = catalog.listTables(namespace.toArray) + identifiers.map { identifier => + if (pattern.map(StringUtils.filterPattern( + Seq(identifier.name()), _).nonEmpty).getOrElse(true)) { + if (!isExtended) { + rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), + isTempView(identifier)) + } else { + val table = catalog.loadTable(identifier) + if (!partitionSpec.isEmpty && table.isPartitionable) { + // Show the information of partitions. + rows += toCatalystRow( + identifier.namespace().quoted, + identifier.name(), + isTempView(identifier), + s"${extendedPartition(identifier, table.asPartitionable, partitionSpec)}") + } else { + // Show the information of tables. + rows += toCatalystRow( + identifier.namespace().quoted, + identifier.name(), + isTempView(identifier), + s"${extendedTable(identifier, table)}") + } + } } } @@ -53,4 +86,113 @@ case class ShowTablesExec( case _ => false } } + + private def extendedTable(identifier: Identifier, table: Table): String = { + val results = new mutable.LinkedHashMap[String, String]() + + if (!identifier.namespace().isEmpty) { + results.put("Namespace", identifier.namespace().quoted) + } + results.put("Table", identifier.name()) + val tableType = if (table.properties().containsKey(TableCatalog.PROP_EXTERNAL)) { + CatalogTableType.EXTERNAL + } else { + CatalogTableType.MANAGED + } + results.put("Type", tableType.name) + + CatalogV2Util.TABLE_RESERVED_PROPERTIES + .filterNot(_ == TableCatalog.PROP_EXTERNAL) + .foreach(propKey => { + if (table.properties.containsKey(propKey)) { + results.put(propKey.capitalize, table.properties.get(propKey)) + } + }) + + val properties = + conf.redactOptions(table.properties.asScala.toMap).toList + .filter(kv => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(kv._1)) + .sortBy(_._1).map { + case (key, value) => key + "=" + value + }.mkString("[", ",", "]") + if (table.properties().isEmpty) { + results.put("Table Properties", properties.mkString("[", ", ", "]")) + } + + // Partition Provider & Partition Columns + // TODO check + if (table.isPartitionable && !table.asPartitionable.partitionSchema().isEmpty) { + results.put("Partition Provider", "Catalog") + results.put("Partition Columns", table.asPartitionable.partitionSchema().map( + field => quoteIdentifier(field.name)).mkString(", ")) + } + + if (table.schema().nonEmpty) results.put("Schema", table.schema().treeString) + + results.map { case ((key, value)) => + if (value.isEmpty) key else s"$key: $value" + }.mkString("", "\n", "") + } + + private def extendedPartition( + identifier: Identifier, + partitionTable: SupportsPartitionManagement, + partitionSpec: Option[TablePartitionSpec]): String = { + val results = new mutable.LinkedHashMap[String, String]() + + // "Partition Values" + val partitionSchema = partitionTable.partitionSchema() + val normalizedSpec = normalizePartitionSpec( + partitionSpec.get, + partitionSchema, + partitionTable.name(), + conf.resolver) + requireExactMatchedPartitionSpec(identifier.toString, + normalizedSpec, partitionSchema.fieldNames) + + val partitionNames = normalizedSpec.keySet + val (names, ident) = (partitionSchema.map(_.name), + convertToPartIdent(normalizedSpec, partitionSchema)) + val partitionIdentifiers = partitionTable.listPartitionIdentifiers(names.toArray, ident) + partitionIdentifiers.length match { + case 0 => + throw QueryExecutionErrors.notExistPartitionError( + identifier.toString, ident, partitionSchema) + case len if (len > 1) => + throw QueryExecutionErrors.showTableExtendedMultiPartitionUnsupportedError( + identifier.toString) + case _ => // do nothing + } + val partitionIdentifier = partitionIdentifiers.head + val len = partitionSchema.length + val partitions = new Array[String](len) + val timeZoneId = conf.sessionLocalTimeZone + var i = 0 + while (i < len) { + val dataType = partitionSchema(i).dataType + val partValueUTF8String = + Cast(Literal(partitionIdentifier.get(i, dataType), dataType), + StringType, Some(timeZoneId)).eval() + val partValueStr = if (partValueUTF8String == null) "null" else partValueUTF8String.toString + partitions(i) = escapePathName(partitionSchema(i).name) + "=" + escapePathName(partValueStr) + i += 1 + } + val partitionValues = partitions.mkString("[", ", ", "]") + results.put("Partition Values", s"${partitionValues}") + + // "Partition Parameters" + val metadata = partitionTable.loadPartitionMetadata(ident) + if (!metadata.isEmpty) { + val metadataValues = metadata.map { case ((key, value)) => + if (value.isEmpty) key else s"$key: $value" + }.mkString("{", ", ", "}") + results.put("Partition Parameters", metadataValues) + } + + // TODO "Created Time", "Last Access", "Partition Statistics" + + results.map { case ((key, value)) => + if (value.isEmpty) key else s"$key: $value" + }.mkString("", "\n", "\n") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala index d68e1233f7ab..0815bcaa90fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace, UnresolvedPartitionSpec} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.logical.{ShowTableExtended, ShowTables} import org.apache.spark.sql.test.SharedSparkSession @@ -64,20 +64,20 @@ class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession { ShowTableExtended( UnresolvedNamespace(Seq.empty[String]), "*test*", - Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09", "hr" -> "11"))))) + Some(Map("ds" -> "2008-04-09", "hr" -> "11")))) comparePlans( parsePlan(s"SHOW TABLE EXTENDED FROM $catalog.ns1.ns2 LIKE '*test*' " + "PARTITION(ds='2008-04-09')"), ShowTableExtended( UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*", - Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09"))))) + Some(Map("ds" -> "2008-04-09")))) comparePlans( parsePlan(s"SHOW TABLE EXTENDED IN $catalog.ns1.ns2 LIKE '*test*' " + "PARTITION(ds='2008-04-09')"), ShowTableExtended( UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*", - Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09"))))) + Some(Map("ds" -> "2008-04-09")))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index 6dc8a050f9ca..8a5c925a4c2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.command.v2 -import org.apache.spark.sql.{AnalysisException, Row} -import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchPartitionException} import org.apache.spark.sql.execution.command +import org.apache.spark.util.Utils /** * The class contains tests for the `SHOW TABLES` command to check V2 table catalogs. @@ -49,52 +50,113 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase } } - // The test fails for V1 catalog with the error: - // org.apache.spark.sql.AnalysisException: - // The namespace in session catalog must have exactly one name part: spark_catalog.ns1.ns2.tbl - test("SHOW TABLE EXTENDED not valid v1 database") { - def testV1CommandNamespace(sqlCommand: String, namespace: String): Unit = { - val e = intercept[AnalysisException] { - sql(sqlCommand) - } - assert(e.message.contains(s"SHOW TABLE EXTENDED is not supported for v2 tables")) - } + test("show table in a not existing namespace") { + val msg = intercept[NoSuchNamespaceException] { + runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq()) + }.getMessage + assert(msg.matches("(Database|Namespace) 'unknown' not found")) + } - val namespace = s"$catalog.ns1.ns2" + test("SHOW TABLE EXTENDED for v2 tables") { + val namespace = "ns1.ns2" val table = "tbl" - withTable(s"$namespace.$table") { - sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " + + withNamespaceAndTable(s"$namespace", s"$table", s"$catalog") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) " + s"$defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id=1)") + + val result1 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE 'tb*'") + assert(result1.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result1.collect()(0).length == 4) + assert(result1.collect()(0)(3) == + s"""Namespace: ns1.ns2 + |Table: tbl + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: `id` + |Schema: root + | |-- id: long (nullable = true) + | |-- data: string (nullable = true) + |""".stripMargin) + + val result2 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") + assert(result2.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result2.collect()(0).length == 4) + assert(result2.collect()(0)(3) === + s"""Namespace: ns1.ns2 + |Table: tbl + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: `id` + |Schema: root + | |-- id: long (nullable = true) + | |-- data: string (nullable = true) + |""".stripMargin) + + val result3 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE 'tb*' PARTITION(id = 1)") + assert(result3.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result3.collect()(0).length == 4) + assert(result3.collect()(0)(3) === + """Partition Values: [id=1] + |""".stripMargin) + + val result4 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*' PARTITION(id = 1)") + assert(result4.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result4.collect()(0).length == 4) + assert(result4.collect()(0)(3) === + """Partition Values: [id=1] + |""".stripMargin) - testV1CommandNamespace(s"SHOW TABLE EXTENDED FROM $namespace LIKE 'tb*'", - namespace) - testV1CommandNamespace(s"SHOW TABLE EXTENDED IN $namespace LIKE 'tb*'", - namespace) - testV1CommandNamespace("SHOW TABLE EXTENDED " + - s"FROM $namespace LIKE 'tb*' PARTITION(id=1)", - namespace) - testV1CommandNamespace("SHOW TABLE EXTENDED " + - s"IN $namespace LIKE 'tb*' PARTITION(id=1)", - namespace) + sql(s"ALTER TABLE $tbl SET LOCATION 's3://bucket/path'") + val result5 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") + assert(result5.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result5.collect()(0).length == 4) + assert(result5.collect()(0)(1) === "tbl") + assert(result5.collect()(0)(3) === + s"""Namespace: ns1.ns2 + |Table: tbl + |Type: MANAGED + |Location: s3://bucket/path + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Schema: root + | |-- id: long (nullable = true) + | |-- data: string (nullable = true) + |""".stripMargin) } } - // TODO(SPARK-33393): Support SHOW TABLE EXTENDED in DSv2 - test("SHOW TABLE EXTENDED: an existing table") { - val table = "people" - withTable(s"$catalog.$table") { - sql(s"CREATE TABLE $catalog.$table (name STRING, id INT) $defaultUsing") - val errMsg = intercept[AnalysisException] { - sql(s"SHOW TABLE EXTENDED FROM $catalog LIKE '*$table*'").collect() + test("SHOW TABLE EXTENDED in a not existing partition") { + val namespace = "ns1.ns2" + val table = "tbl" + withNamespaceAndTable(s"$namespace", s"$table", s"$catalog") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id=1)") + val errMsg = intercept[NoSuchPartitionException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*' PARTITION(id = 2)") }.getMessage - assert(errMsg.contains("SHOW TABLE EXTENDED is not supported for v2 tables")) + assert(errMsg === "Partition not found in table ns1.ns2.tbl: 2 -> id") } } - test("show table in a not existing namespace") { - val msg = intercept[NoSuchNamespaceException] { - runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq()) - }.getMessage - assert(msg.matches("(Database|Namespace) 'unknown' not found")) + test("SHOW TABLE EXTENDED in a not existing table") { + val table = "people" + withTable(s"$catalog.$table") { + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog LIKE '*$table*'") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect().isEmpty) + } } } From cbe7433c5ac2cadb18d8e6c6f7706b84ff0d3e67 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sat, 20 Aug 2022 21:28:37 +0800 Subject: [PATCH 02/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 2 ++ .../spark/sql/execution/datasources/v2/ShowTablesExec.scala | 1 + 2 files changed, 3 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 2f8ba1b18efd..3a2fe6b88403 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -26,10 +26,12 @@ import java.time.{DateTimeException, LocalDate} import java.time.temporal.ChronoField import java.util.ConcurrentModificationException import java.util.concurrent.TimeoutException + import com.fasterxml.jackson.core.{JsonParser, JsonToken} import org.apache.hadoop.fs.{FileAlreadyExistsException, FileStatus, Path} import org.apache.hadoop.fs.permission.FsPermission import org.codehaus.commons.compiler.{CompileException, InternalCompilerException} + import org.apache.spark._ import org.apache.spark.executor.CommitDeniedException import org.apache.spark.launcher.SparkLauncher diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 4e6d12c16d7a..93fee16cfd78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import scala.collection.convert.ImplicitConversions.`map AsScala` import scala.collection.mutable import scala.collection.mutable.ArrayBuffer + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec.convertToPartIdent import org.apache.spark.sql.catalyst.catalog.CatalogTableType From 023414b37119e77ac4e9bcd82a51b4458cbf6d47 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sat, 20 Aug 2022 21:47:13 +0800 Subject: [PATCH 03/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v --- .../spark/sql/execution/datasources/v2/ShowTablesExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 93fee16cfd78..d4d994979ba8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} -import org.apache.spark.sql.catalyst.util.{StringUtils, quoteIdentifier} +import org.apache.spark.sql.catalyst.util.{quoteIdentifier, StringUtils} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, SupportsPartitionManagement, Table, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryExecutionErrors From 06977733abd4a6b0d3549ee5f8264cfe0b4a2e84 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sun, 28 Aug 2022 18:21:43 +0800 Subject: [PATCH 04/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../sql/catalyst/analysis/ResolvePartitionSpec.scala | 6 +++++- .../sql/catalyst/analysis/v2ResolutionPlans.scala | 9 +++++++++ .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 4 +++- .../spark/sql/catalyst/plans/logical/v2Commands.scala | 2 +- .../sql/catalyst/analysis/ResolveSessionCatalog.scala | 10 ++++++++-- .../datasources/v2/DataSourceV2Strategy.scala | 7 ++++--- .../sql/execution/command/ShowTablesParserSuite.scala | 8 ++++---- 7 files changed, 34 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index e09991cb2b90..19a576a035c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2PartitionCommand} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ShowTableExtended, V2PartitionCommand} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND import org.apache.spark.sql.catalyst.util.CharVarcharUtils @@ -49,6 +49,10 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { } case _ => command } + case s @ ShowTableExtended(_, _, partitionSpec @ Some(UnresolvedPartitionSpec(_, _)), _) => + val extractPartitionSpec = new ExtractPartitionSpec( + partitionSpec.get.asInstanceOf[UnresolvedPartitionSpec]) + s.copy(partitionSpec = Some(extractPartitionSpec)) } private def resolvePartitionSpec( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 6095d812d668..a998838f6666 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -91,6 +91,15 @@ case class UnresolvedPartitionSpec( override lazy val resolved = false } +case class ExtractPartitionSpec( + spec: Map[String, String], + location: Option[String] = None) extends PartitionSpec { + + def this(u: UnresolvedPartitionSpec) = this(u.spec, u.location) + + override lazy val resolved = true +} + sealed trait FieldName extends LeafExpression with Unevaluable { def name: Seq[String] override def dataType: DataType = throw new IllegalStateException( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 2ebb69ebabb5..281146d3a38f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3715,7 +3715,9 @@ class AstBuilder extends SqlBaseParserBaseVisitor[AnyRef] with SQLConfHelper wit override def visitShowTableExtended( ctx: ShowTableExtendedContext): LogicalPlan = withOrigin(ctx) { val multiPart = Option(ctx.multipartIdentifier).map(visitMultipartIdentifier) - val partitionKeys = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) + val partitionKeys = Option(ctx.partitionSpec).map { specCtx => + UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) + } ShowTableExtended( UnresolvedNamespace(multiPart.getOrElse(Seq.empty[String])), string(ctx.pattern), 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 2affd3fa851b..e5bc96eab8a3 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 @@ -706,7 +706,7 @@ object ShowTables { case class ShowTableExtended( namespace: LogicalPlan, pattern: String, - partitionSpec: Option[TablePartitionSpec], + partitionSpec: Option[PartitionSpec], override val output: Seq[Attribute] = ShowTableExtended.getOutputAttrs) extends UnaryCommand { override def child: LogicalPlan = namespace override protected def withNewChildInternal(newChild: LogicalPlan): ShowTableExtended = 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 ee01cf41980f..9fac23f702c9 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 @@ -234,7 +234,8 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case ShowTableExtended( DatabaseInSessionCatalog(db), pattern, - partitionSpec, + partitionSpec @ (None | Some(UnresolvedPartitionSpec(_, _)) | + Some(ExtractPartitionSpec(_, _))), output) => val newOutput = if (conf.getConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA)) { assert(output.length == 4) @@ -242,7 +243,12 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } else { output } - ShowTablesCommand(Some(db), Some(pattern), newOutput, true, partitionSpec) + val tablePartitionSpec = partitionSpec match { + case Some(UnresolvedPartitionSpec(spec, _)) => Some(spec) + case Some(ExtractPartitionSpec(spec, _)) => Some(spec) + case _ => None + } + ShowTablesCommand(Some(db), Some(pattern), newOutput, true, tablePartitionSpec) // ANALYZE TABLE works on permanent views if the views are cached. case AnalyzeTable(ResolvedV1TableOrViewIdentifier(ident), partitionSpec, noScan) => 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 675cdff18b9f..71e4cea95ae5 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 @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, Strategy} -import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable} +import org.apache.spark.sql.catalyst.analysis.{ExtractPartitionSpec, ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable} import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning, Expression, NamedExpression, Not, Or, PredicateHelper, SubqueryExpression} @@ -371,15 +371,16 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case ShowTableExtended( ResolvedNamespace(catalog, ns), pattern, - partitionSpec, + partitionSpec @ (None | Some(ExtractPartitionSpec(_, _))), output) => + val tablePartitionSpec = partitionSpec.map(_.asInstanceOf[ExtractPartitionSpec].spec) ShowTablesExec( output, catalog.asTableCatalog, ns, Some(pattern), true, - partitionSpec) :: Nil + tablePartitionSpec) :: Nil case SetCatalogAndNamespace(ResolvedNamespace(catalog, ns)) => val catalogManager = session.sessionState.catalogManager diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala index 0815bcaa90fe..d68e1233f7ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace, UnresolvedPartitionSpec} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.logical.{ShowTableExtended, ShowTables} import org.apache.spark.sql.test.SharedSparkSession @@ -64,20 +64,20 @@ class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession { ShowTableExtended( UnresolvedNamespace(Seq.empty[String]), "*test*", - Some(Map("ds" -> "2008-04-09", "hr" -> "11")))) + Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09", "hr" -> "11"))))) comparePlans( parsePlan(s"SHOW TABLE EXTENDED FROM $catalog.ns1.ns2 LIKE '*test*' " + "PARTITION(ds='2008-04-09')"), ShowTableExtended( UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*", - Some(Map("ds" -> "2008-04-09")))) + Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09"))))) comparePlans( parsePlan(s"SHOW TABLE EXTENDED IN $catalog.ns1.ns2 LIKE '*test*' " + "PARTITION(ds='2008-04-09')"), ShowTableExtended( UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*", - Some(Map("ds" -> "2008-04-09")))) + Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09"))))) } } From 98761208620e059a9143a8f8f3a6f6b921f81769 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sun, 5 Feb 2023 17:17:58 +0800 Subject: [PATCH 05/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../sql/execution/command/v2/ShowTablesSuite.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index 4443c7bc49c7..a001e9fbf5a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -145,10 +145,16 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase sql(s"CREATE TABLE $tbl (id bigint, data string) " + s"$defaultUsing PARTITIONED BY (id)") sql(s"ALTER TABLE $tbl ADD PARTITION (id=1)") - val errMsg = intercept[NoSuchPartitionException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*' PARTITION(id = 2)") - }.getMessage - assert(errMsg === "Partition not found in table ns1.ns2.tbl: 2 -> id") + checkError( + exception = intercept[NoSuchPartitionException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*' PARTITION(id = 2)") + }, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "PARTITION (`id` = 2)", + "tableName" -> "`ns1`.`ns2`.`tbl`" + ) + ) } } From 7d71eb0a547cb57cf8779405d598a02c4858d84f Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 20 Feb 2023 17:08:50 +0800 Subject: [PATCH 06/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../analysis/ResolvePartitionSpec.scala | 15 ++-- .../catalyst/analysis/v2ResolutionPlans.scala | 9 -- .../analysis/ResolveSessionCatalog.scala | 9 +- .../datasources/v2/DataSourceV2Strategy.scala | 17 ++-- .../datasources/v2/ShowTablesExec.scala | 82 ++++++++----------- .../command/v2/ShowTablesSuite.scala | 74 +++++++++-------- 6 files changed, 88 insertions(+), 118 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index 9817161e7c96..3e7c5788665c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -24,7 +24,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ShowTableExtend import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND import org.apache.spark.sql.catalyst.util.CharVarcharUtils -import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsPartitionManagement, TableCatalog} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.PartitioningUtils.{castPartitionSpec, normalizePartitionSpec, requireExactMatchedPartitionSpec} @@ -50,10 +51,14 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { } case _ => command } - case s @ ShowTableExtended(_, _, partitionSpec @ Some(UnresolvedPartitionSpec(_, _)), _) => - val extractPartitionSpec = new ExtractPartitionSpec( - partitionSpec.get.asInstanceOf[UnresolvedPartitionSpec]) - s.copy(partitionSpec = Some(extractPartitionSpec)) + case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, + namespace), pattern, _, _) + if command.childrenResolved && !command.resolved => command.transformExpressions { + case partSpecs: UnresolvedPartitionSpec => + val table = catalog.loadTable(Identifier.of(namespace.toArray, pattern)) + val partitionSchema = table.asPartitionable.partitionSchema() + resolvePartitionSpec(table.name, partSpecs, partitionSchema, false) + } } private def resolvePartitionSpec( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 193cdc25e3b4..e6be5c239555 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -93,15 +93,6 @@ case class UnresolvedPartitionSpec( override lazy val resolved = false } -case class ExtractPartitionSpec( - spec: Map[String, String], - location: Option[String] = None) extends PartitionSpec { - - def this(u: UnresolvedPartitionSpec) = this(u.spec, u.location) - - override lazy val resolved = true -} - sealed trait FieldName extends LeafExpression with Unevaluable { def name: Seq[String] override def dataType: DataType = throw new IllegalStateException( 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 899bf42ac03d..7b2d5015840c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -256,8 +256,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case ShowTableExtended( DatabaseInSessionCatalog(db), pattern, - partitionSpec @ (None | Some(UnresolvedPartitionSpec(_, _)) | - Some(ExtractPartitionSpec(_, _))), + partitionSpec @ (None | Some(UnresolvedPartitionSpec(_, _))), output) => val newOutput = if (conf.getConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA)) { assert(output.length == 4) @@ -265,11 +264,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } else { output } - val tablePartitionSpec = partitionSpec match { - case Some(UnresolvedPartitionSpec(spec, _)) => Some(spec) - case Some(ExtractPartitionSpec(spec, _)) => Some(spec) - case _ => None - } + val tablePartitionSpec = partitionSpec.map(_.asInstanceOf[UnresolvedPartitionSpec].spec) ShowTablesCommand(Some(db), Some(pattern), newOutput, true, tablePartitionSpec) // ANALYZE TABLE works on permanent views if the views are cached. 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 7899e8cf5839..392a6e5c5cf1 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 @@ -24,7 +24,7 @@ import org.apache.commons.lang3.StringUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, Strategy} -import org.apache.spark.sql.catalyst.analysis.{ExtractPartitionSpec, ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable} +import org.apache.spark.sql.catalyst.analysis.{ResolvedIdentifier, ResolvedNamespace, ResolvedPartitionSpec, ResolvedTable} import org.apache.spark.sql.catalyst.catalog.CatalogUtils import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.{And, Attribute, DynamicPruning, Expression, NamedExpression, Not, Or, PredicateHelper, SubqueryExpression} @@ -396,18 +396,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat ShowTablesExec(output, catalog.asTableCatalog, ns, pattern) :: Nil case ShowTableExtended( - ResolvedNamespace(catalog, ns), - pattern, - partitionSpec @ (None | Some(ExtractPartitionSpec(_, _))), + ResolvedNamespace(catalog, ns), pattern, + partitionSpec @ (None | Some(_: ResolvedPartitionSpec)), output) => - val tablePartitionSpec = partitionSpec.map(_.asInstanceOf[ExtractPartitionSpec].spec) - ShowTablesExec( - output, - catalog.asTableCatalog, - ns, - Some(pattern), - true, - tablePartitionSpec) :: Nil + ShowTablesExec(output, catalog.asTableCatalog, ns, Some(pattern), true, + partitionSpec.map(_.asInstanceOf[ResolvedPartitionSpec])) :: Nil case SetCatalogAndNamespace(ResolvedNamespace(catalog, ns)) => val catalogManager = session.sessionState.catalogManager diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index d4d994979ba8..3517562a8e4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -23,9 +23,8 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec.convertToPartIdent +import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec import org.apache.spark.sql.catalyst.catalog.CatalogTableType -import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, StringUtils} @@ -35,7 +34,6 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper import org.apache.spark.sql.types.StringType -import org.apache.spark.sql.util.PartitioningUtils.{normalizePartitionSpec, requireExactMatchedPartitionSpec} /** * Physical plan node for showing tables. @@ -46,36 +44,34 @@ case class ShowTablesExec( namespace: Seq[String], pattern: Option[String], isExtended: Boolean = false, - partitionSpec: Option[TablePartitionSpec] = None) extends V2CommandExec with LeafExecNode { + partitionSpec: Option[ResolvedPartitionSpec] = None) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - val identifiers = catalog.listTables(namespace.toArray) - identifiers.map { identifier => - if (pattern.map(StringUtils.filterPattern( - Seq(identifier.name()), _).nonEmpty).getOrElse(true)) { - if (!isExtended) { - rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), - isTempView(identifier)) - } else { - val table = catalog.loadTable(identifier) - if (!partitionSpec.isEmpty && table.isPartitionable) { - // Show the information of partitions. - rows += toCatalystRow( - identifier.namespace().quoted, - identifier.name(), - isTempView(identifier), - s"${extendedPartition(identifier, table.asPartitionable, partitionSpec)}") + if (partitionSpec.isEmpty) { + // Show the information of tables. + val identifiers = catalog.listTables(namespace.toArray) + identifiers.map { identifier => + if (pattern.map(StringUtils.filterPattern( + Seq(identifier.name()), _).nonEmpty).getOrElse(true)) { + val isTemp = isTempView(identifier) + if (isExtended) { + val table = catalog.loadTable(identifier) + val information = extendedTable(identifier, table) + rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), isTemp, + s"$information\n") } else { - // Show the information of tables. - rows += toCatalystRow( - identifier.namespace().quoted, - identifier.name(), - isTempView(identifier), - s"${extendedTable(identifier, table)}") + rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), isTemp) } } } + } else { + // Show the information of partitions. + val identifier = Identifier.of(namespace.toArray, pattern.get) + val table = catalog.loadTable(identifier) + val isTemp = isTempView(identifier) + val information = extendedPartition(identifier, table.asPartitionable, partitionSpec.get) + rows += toCatalystRow(namespace.quoted, table.name(), isTemp, s"$information\n") } rows.toSeq @@ -121,7 +117,6 @@ case class ShowTablesExec( } // Partition Provider & Partition Columns - // TODO check if (table.isPartitionable && !table.asPartitionable.partitionSchema().isEmpty) { results.put("Partition Provider", "Catalog") results.put("Partition Columns", table.asPartitionable.partitionSchema().map( @@ -130,7 +125,7 @@ case class ShowTablesExec( if (table.schema().nonEmpty) results.put("Schema", table.schema().treeString) - results.map { case ((key, value)) => + results.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" }.mkString("", "\n", "") } @@ -138,53 +133,40 @@ case class ShowTablesExec( private def extendedPartition( identifier: Identifier, partitionTable: SupportsPartitionManagement, - partitionSpec: Option[TablePartitionSpec]): String = { + resolvedPartitionSpec: ResolvedPartitionSpec): String = { val results = new mutable.LinkedHashMap[String, String]() // "Partition Values" val partitionSchema = partitionTable.partitionSchema() - val normalizedSpec = normalizePartitionSpec( - partitionSpec.get, - partitionSchema, - partitionTable.name(), - conf.resolver) - requireExactMatchedPartitionSpec(identifier.toString, - normalizedSpec, partitionSchema.fieldNames) - - val partitionNames = normalizedSpec.keySet - val (names, ident) = (partitionSchema.map(_.name), - convertToPartIdent(normalizedSpec, partitionSchema)) + val (names, ident) = (resolvedPartitionSpec.names, resolvedPartitionSpec.ident) val partitionIdentifiers = partitionTable.listPartitionIdentifiers(names.toArray, ident) partitionIdentifiers.length match { case 0 => throw QueryExecutionErrors.notExistPartitionError( identifier.toString, ident, partitionSchema) - case len if (len > 1) => + case len if len > 1 => throw QueryExecutionErrors.showTableExtendedMultiPartitionUnsupportedError( identifier.toString) case _ => // do nothing } - val partitionIdentifier = partitionIdentifiers.head + val row = partitionIdentifiers.head val len = partitionSchema.length val partitions = new Array[String](len) val timeZoneId = conf.sessionLocalTimeZone - var i = 0 - while (i < len) { + for (i <- 0 until len) { val dataType = partitionSchema(i).dataType val partValueUTF8String = - Cast(Literal(partitionIdentifier.get(i, dataType), dataType), - StringType, Some(timeZoneId)).eval() + Cast(Literal(row.get(i, dataType), dataType), StringType, Some(timeZoneId)).eval() val partValueStr = if (partValueUTF8String == null) "null" else partValueUTF8String.toString partitions(i) = escapePathName(partitionSchema(i).name) + "=" + escapePathName(partValueStr) - i += 1 } val partitionValues = partitions.mkString("[", ", ", "]") - results.put("Partition Values", s"${partitionValues}") + results.put("Partition Values", s"$partitionValues") // "Partition Parameters" val metadata = partitionTable.loadPartitionMetadata(ident) if (!metadata.isEmpty) { - val metadataValues = metadata.map { case ((key, value)) => + val metadataValues = metadata.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" }.mkString("{", ", ", "}") results.put("Partition Parameters", metadataValues) @@ -192,7 +174,7 @@ case class ShowTablesExec( // TODO "Created Time", "Last Access", "Partition Statistics" - results.map { case ((key, value)) => + results.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" }.mkString("", "\n", "\n") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index a001e9fbf5a0..362ca7d1d563 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -59,13 +59,41 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase parameters = Map("schemaName" -> "`unknown`")) } + test("SHOW TABLE EXTENDED in a not existing table") { + val table = "people" + withTable(s"$catalog.$table") { + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog LIKE '*$table*'") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect().isEmpty) + } + } + + test("SHOW TABLE EXTENDED in a not existing partition") { + val namespace = "ns1.ns2" + val table = "tbl" + withNamespaceAndTable(s"$namespace", s"$table", s"$catalog") { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") + checkError( + exception = intercept[NoSuchPartitionException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tbl' PARTITION(id = 2)") + }, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "PARTITION (`id` = 2)", + "tableName" -> "`ns1`.`ns2`.`tbl`" + ) + ) + } + } + test("SHOW TABLE EXTENDED for v2 tables") { val namespace = "ns1.ns2" val table = "tbl" withNamespaceAndTable(s"$namespace", s"$table", s"$catalog") { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id)") - sql(s"ALTER TABLE $tbl ADD PARTITION (id=1)") + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") val result1 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE 'tb*'") assert(result1.schema.fieldNames === @@ -82,6 +110,7 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase |Schema: root | |-- id: long (nullable = true) | |-- data: string (nullable = true) + | |""".stripMargin) val result2 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") @@ -99,23 +128,27 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase |Schema: root | |-- id: long (nullable = true) | |-- data: string (nullable = true) + | |""".stripMargin) val result3 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - s"LIKE 'tb*' PARTITION(id = 1)") + "LIKE 'tbl' PARTITION(id = 1)") assert(result3.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) assert(result3.collect()(0).length == 4) assert(result3.collect()(0)(3) === """Partition Values: [id=1] + | |""".stripMargin) - val result4 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*' PARTITION(id = 1)") + val result4 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + + "LIKE 'tbl' PARTITION(id = 1)") assert(result4.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) assert(result4.collect()(0).length == 4) assert(result4.collect()(0)(3) === """Partition Values: [id=1] + | |""".stripMargin) sql(s"ALTER TABLE $tbl SET LOCATION 's3://bucket/path'") @@ -134,37 +167,8 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase |Schema: root | |-- id: long (nullable = true) | |-- data: string (nullable = true) + | |""".stripMargin) } } - - test("SHOW TABLE EXTENDED in a not existing partition") { - val namespace = "ns1.ns2" - val table = "tbl" - withNamespaceAndTable(s"$namespace", s"$table", s"$catalog") { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id)") - sql(s"ALTER TABLE $tbl ADD PARTITION (id=1)") - checkError( - exception = intercept[NoSuchPartitionException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*' PARTITION(id = 2)") - }, - errorClass = "PARTITIONS_NOT_FOUND", - parameters = Map( - "partitionList" -> "PARTITION (`id` = 2)", - "tableName" -> "`ns1`.`ns2`.`tbl`" - ) - ) - } - } - - test("SHOW TABLE EXTENDED in a not existing table") { - val table = "people" - withTable(s"$catalog.$table") { - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog LIKE '*$table*'") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect().isEmpty) - } - } } From 3c3651a958f66e0da13039465bcfe86ff0aebb8b Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 20 Feb 2023 19:34:03 +0800 Subject: [PATCH 07/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../sql/catalyst/analysis/ResolvePartitionSpec.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index 3e7c5788665c..66c7bccf2c9e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.connector.catalog.{Identifier, SupportsPartitionManagement, TableCatalog} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.PartitioningUtils.{castPartitionSpec, normalizePartitionSpec, requireExactMatchedPartitionSpec} @@ -56,8 +55,11 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { if command.childrenResolved && !command.resolved => command.transformExpressions { case partSpecs: UnresolvedPartitionSpec => val table = catalog.loadTable(Identifier.of(namespace.toArray, pattern)) - val partitionSchema = table.asPartitionable.partitionSchema() - resolvePartitionSpec(table.name, partSpecs, partitionSchema, false) + table match { + case s: SupportsPartitionManagement => + resolvePartitionSpec(table.name, partSpecs, s.partitionSchema(), false) + case _ => partSpecs + } } } From b1cc56c394da1592012a4735096acb39f6dceee6 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 21 Feb 2023 11:28:47 +0800 Subject: [PATCH 08/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../sql/errors/QueryExecutionErrors.scala | 5 - .../v2/DataSourceV2Implicits.scala | 5 + .../datasources/v2/ShowTablesExec.scala | 14 +- .../command/v2/ShowTablesSuite.scala | 213 ++++++++++++++++-- 4 files changed, 199 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index b9fc3ba5ec40..40b2a11833bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -2792,11 +2792,6 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase { "identifier" -> toSQLId(tableId.nameParts))) } - def showTableExtendedMultiPartitionUnsupportedError(tableName: String): Throwable = { - new UnsupportedOperationException( - s"The table $tableName does not support show table extended of multiple partition.") - } - def notExistPartitionError( tableName: String, partitionIdent: InternalRow, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index 713d210c1606..6a02e44f44c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -30,6 +30,11 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Implicits { implicit class TableHelper(table: Table) { + def isReadable: Boolean = table match { + case _: SupportsRead => true + case _ => false + } + def asReadable: SupportsRead = { table match { case support: SupportsRead => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 3517562a8e4d..0bfc24a51947 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -123,7 +123,9 @@ case class ShowTablesExec( field => quoteIdentifier(field.name)).mkString(", ")) } - if (table.schema().nonEmpty) results.put("Schema", table.schema().treeString) + if (table.isReadable) { + if (table.schema().nonEmpty) results.put("Schema", table.schema().treeString) + } results.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" @@ -140,14 +142,8 @@ case class ShowTablesExec( val partitionSchema = partitionTable.partitionSchema() val (names, ident) = (resolvedPartitionSpec.names, resolvedPartitionSpec.ident) val partitionIdentifiers = partitionTable.listPartitionIdentifiers(names.toArray, ident) - partitionIdentifiers.length match { - case 0 => - throw QueryExecutionErrors.notExistPartitionError( - identifier.toString, ident, partitionSchema) - case len if len > 1 => - throw QueryExecutionErrors.showTableExtendedMultiPartitionUnsupportedError( - identifier.toString) - case _ => // do nothing + if (partitionIdentifiers.length == 0) { + throw QueryExecutionErrors.notExistPartitionError(identifier.toString, ident, partitionSchema) } val row = partitionIdentifiers.head val len = partitionSchema.length diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index 362ca7d1d563..dc85fcc2a918 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command.v2 -import org.apache.spark.sql.Row +import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchPartitionException} import org.apache.spark.sql.execution.command import org.apache.spark.util.Utils @@ -59,8 +59,17 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase parameters = Map("schemaName" -> "`unknown`")) } - test("SHOW TABLE EXTENDED in a not existing table") { - val table = "people" + test("show table extended in a not existing namespace") { + checkError( + exception = intercept[NoSuchNamespaceException] { + sql(s"SHOW TABLE EXTENDED FROM $catalog.unknown LIKE '*tbl*'") + }, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`unknown`")) + } + + test("show table extended in a not existing table") { + val table = "nonexist" withTable(s"$catalog.$table") { val result = sql(s"SHOW TABLE EXTENDED FROM $catalog LIKE '*$table*'") assert(result.schema.fieldNames === @@ -69,15 +78,64 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase } } - test("SHOW TABLE EXTENDED in a not existing partition") { + test("show table extended in non-partitioned table") { + val namespace = "ns1.ns2" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") + checkError( + exception = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") + }, + errorClass = "_LEGACY_ERROR_TEMP_1231", + parameters = Map("key" -> "id", "tblName" -> s"$catalog.$namespace.$table") + ) + } + } + + test("show table extended in multi-partition table") { val namespace = "ns1.ns2" val table = "tbl" - withNamespaceAndTable(s"$namespace", s"$table", s"$catalog") { tbl => + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id1, id2)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(0) === namespace) + assert(result.collect()(0)(1) === s"$catalog.$namespace.$table") + assert(result.collect()(0)(2) === false) + assert(result.collect()(0)(3) === + """Partition Values: [id1=1, id2=2] + | + |""".stripMargin) + + checkError( + exception = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)") + }, + errorClass = "_LEGACY_ERROR_TEMP_1232", + parameters = Map( + "specKeys" -> "id1", + "partitionColumnNames" -> "id1, id2", + "tableName" -> s"$catalog.$namespace.$table") + ) + } + } + + test("show table extended in a not existing partition") { + val namespace = "ns1.ns2" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") checkError( exception = intercept[NoSuchPartitionException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tbl' PARTITION(id = 2)") + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 2)") }, errorClass = "PARTITIONS_NOT_FOUND", parameters = Map( @@ -88,14 +146,14 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase } } - test("SHOW TABLE EXTENDED for v2 tables") { + test("show table extended for v2 tables") { val namespace = "ns1.ns2" val table = "tbl" - withNamespaceAndTable(s"$namespace", s"$table", s"$catalog") { tbl => + withNamespaceAndTable(namespace, table, catalog) { tbl => sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") - val result1 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE 'tb*'") + val result1 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table'") assert(result1.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) assert(result1.collect()(0).length == 4) @@ -113,11 +171,11 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase | |""".stripMargin) - val result2 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") + val result2 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table'") assert(result2.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) assert(result2.collect()(0).length == 4) - assert(result2.collect()(0)(3) === + assert(result2.collect()(0)(3) == s"""Namespace: ns1.ns2 |Table: tbl |Type: MANAGED @@ -131,33 +189,69 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase | |""".stripMargin) - val result3 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - "LIKE 'tbl' PARTITION(id = 1)") + val result3 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE 'tb*'") assert(result3.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) assert(result3.collect()(0).length == 4) - assert(result3.collect()(0)(3) === - """Partition Values: [id=1] - | - |""".stripMargin) + assert(result3.collect()(0)(3) == + s"""Namespace: ns1.ns2 + |Table: tbl + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: `id` + |Schema: root + | |-- id: long (nullable = true) + | |-- data: string (nullable = true) + | + |""".stripMargin) - val result4 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + - "LIKE 'tbl' PARTITION(id = 1)") + val result4 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") assert(result4.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) assert(result4.collect()(0).length == 4) assert(result4.collect()(0)(3) === + s"""Namespace: ns1.ns2 + |Table: tbl + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: `id` + |Schema: root + | |-- id: long (nullable = true) + | |-- data: string (nullable = true) + | + |""".stripMargin) + + val result5 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE '$table' PARTITION(id = 1)") + assert(result5.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result5.collect()(0).length == 4) + assert(result5.collect()(0)(3) === + """Partition Values: [id=1] + | + |""".stripMargin) + + val result6 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + + s"LIKE '$table' PARTITION(id = 1)") + assert(result6.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result6.collect()(0).length == 4) + assert(result6.collect()(0)(3) === """Partition Values: [id=1] | |""".stripMargin) sql(s"ALTER TABLE $tbl SET LOCATION 's3://bucket/path'") - val result5 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") - assert(result5.schema.fieldNames === + val result7 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") + assert(result7.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - assert(result5.collect()(0).length == 4) - assert(result5.collect()(0)(1) === "tbl") - assert(result5.collect()(0)(3) === + assert(result7.collect()(0).length == 4) + assert(result7.collect()(0)(1) === "tbl") + assert(result7.collect()(0)(3) === s"""Namespace: ns1.ns2 |Table: tbl |Type: MANAGED @@ -171,4 +265,75 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase |""".stripMargin) } } + + test("show table extended for v2 multi tables") { + val namespace = "ns1.ns2" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + + val table1 = "tbl1" + val table2 = "tbl2" + withTable(table1, table2) { + sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + + s"$defaultUsing PARTITIONED BY (id1)") + sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + + s"$defaultUsing PARTITIONED BY (id2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") + .sort("tableName") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect().length == 3) + + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(1) === "tbl") + assert(result.collect()(0)(3) === + s"""Namespace: ns1.ns2 + |Table: tbl + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: `id` + |Schema: root + | |-- id: long (nullable = true) + | |-- data: string (nullable = true) + | + |""".stripMargin) + + assert(result.collect()(1).length == 4) + assert(result.collect()(1)(1) === "tbl1") + assert(result.collect()(1)(3) === + s"""Namespace: ns1.ns2 + |Table: tbl1 + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: `id1` + |Schema: root + | |-- id1: long (nullable = true) + | |-- data1: string (nullable = true) + | + |""".stripMargin) + + assert(result.collect()(2).length == 4) + assert(result.collect()(2)(1) === "tbl2") + assert(result.collect()(2)(3) === + s"""Namespace: ns1.ns2 + |Table: tbl2 + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: `id2` + |Schema: root + | |-- id2: long (nullable = true) + | |-- data2: string (nullable = true) + | + |""".stripMargin) + } + } + } } From ab50fb0be8e350d947a393229afc324216e8b5ee Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 21 Feb 2023 13:41:09 +0800 Subject: [PATCH 09/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../spark/sql/execution/command/v2/ShowTablesSuite.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index dc85fcc2a918..eee8cce11bac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -271,7 +271,6 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { tbl => sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") - val table1 = "tbl1" val table2 = "tbl2" withTable(table1, table2) { @@ -285,7 +284,6 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) assert(result.collect().length == 3) - assert(result.collect()(0).length == 4) assert(result.collect()(0)(1) === "tbl") assert(result.collect()(0)(3) === @@ -301,7 +299,6 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase | |-- data: string (nullable = true) | |""".stripMargin) - assert(result.collect()(1).length == 4) assert(result.collect()(1)(1) === "tbl1") assert(result.collect()(1)(3) === @@ -317,7 +314,6 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase | |-- data1: string (nullable = true) | |""".stripMargin) - assert(result.collect()(2).length == 4) assert(result.collect()(2)(1) === "tbl2") assert(result.collect()(2)(3) === From 9c4a9ac89c9a28850e439419fbb425793b181224 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 21 Feb 2023 15:35:18 +0800 Subject: [PATCH 10/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../analysis/ResolvePartitionSpec.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index 66c7bccf2c9e..23e1fefc7b87 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -50,16 +50,15 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { } case _ => command } - case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, - namespace), pattern, _, _) - if command.childrenResolved && !command.resolved => command.transformExpressions { - case partSpecs: UnresolvedPartitionSpec => - val table = catalog.loadTable(Identifier.of(namespace.toArray, pattern)) - table match { - case s: SupportsPartitionManagement => - resolvePartitionSpec(table.name, partSpecs, s.partitionSchema(), false) - case _ => partSpecs - } + case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace), + pattern, _, _) if command.childrenResolved && !command.resolved => + val table = catalog.loadTable(Identifier.of(namespace.toArray, pattern)) + table match { + case s: SupportsPartitionManagement => command.transformExpressions { + case partSpecs: UnresolvedPartitionSpec => + resolvePartitionSpec(table.name, partSpecs, s.partitionSchema(), false) + } + case _ => command } } From 3a296c0bc335e49ce7764bd07990bf736f255e12 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 18 Oct 2023 10:35:13 +0800 Subject: [PATCH 11/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../spark/sql/execution/command/v2/ShowTablesSuite.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index 2f4727304ae0..eee8cce11bac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -50,6 +50,15 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase } } + test("show table in a not existing namespace") { + val e = intercept[NoSuchNamespaceException] { + runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq()) + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`unknown`")) + } + test("show table extended in a not existing namespace") { checkError( exception = intercept[NoSuchNamespaceException] { From f54a832b25672ac0ba972b27aa11a471d00c3123 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 19 Oct 2023 09:06:19 +0800 Subject: [PATCH 12/29] Apply suggestions from code review Co-authored-by: Maxim Gekk --- .../datasources/v2/DataSourceV2Strategy.scala | 3 ++- .../command/v2/ShowTablesSuite.scala | 26 +++++++++---------- 2 files changed, 15 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 44f154047dbd..db1bc5257c7a 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 @@ -405,7 +405,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat ShowTablesExec(output, catalog.asTableCatalog, ns, pattern) :: Nil case ShowTableExtended( - ResolvedNamespace(catalog, ns), pattern, + ResolvedNamespace(catalog, ns), + pattern, partitionSpec @ (None | Some(_: ResolvedPartitionSpec)), output) => ShowTablesExec(output, catalog.asTableCatalog, ns, Some(pattern), true, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index eee8cce11bac..bf399afba1be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -158,8 +158,8 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase Seq("namespace", "tableName", "isTemporary", "information")) assert(result1.collect()(0).length == 4) assert(result1.collect()(0)(3) == - s"""Namespace: ns1.ns2 - |Table: tbl + s"""Namespace: $namespace + |Table: $table |Type: MANAGED |Provider: _ |Owner: ${Utils.getCurrentUserName()} @@ -176,8 +176,8 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase Seq("namespace", "tableName", "isTemporary", "information")) assert(result2.collect()(0).length == 4) assert(result2.collect()(0)(3) == - s"""Namespace: ns1.ns2 - |Table: tbl + s"""Namespace: $namespace + |Table: $table |Type: MANAGED |Provider: _ |Owner: ${Utils.getCurrentUserName()} @@ -194,8 +194,8 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase Seq("namespace", "tableName", "isTemporary", "information")) assert(result3.collect()(0).length == 4) assert(result3.collect()(0)(3) == - s"""Namespace: ns1.ns2 - |Table: tbl + s"""Namespace: $namespace + |Table: $table |Type: MANAGED |Provider: _ |Owner: ${Utils.getCurrentUserName()} @@ -212,8 +212,8 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase Seq("namespace", "tableName", "isTemporary", "information")) assert(result4.collect()(0).length == 4) assert(result4.collect()(0)(3) === - s"""Namespace: ns1.ns2 - |Table: tbl + s"""Namespace: $namespace + |Table: $table |Type: MANAGED |Provider: _ |Owner: ${Utils.getCurrentUserName()} @@ -252,7 +252,7 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase assert(result7.collect()(0).length == 4) assert(result7.collect()(0)(1) === "tbl") assert(result7.collect()(0)(3) === - s"""Namespace: ns1.ns2 + s"""Namespace: $namespace |Table: tbl |Type: MANAGED |Location: s3://bucket/path @@ -300,10 +300,10 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase | |""".stripMargin) assert(result.collect()(1).length == 4) - assert(result.collect()(1)(1) === "tbl1") + assert(result.collect()(1)(1) === table1) assert(result.collect()(1)(3) === s"""Namespace: ns1.ns2 - |Table: tbl1 + |Table: $table1 |Type: MANAGED |Provider: _ |Owner: ${Utils.getCurrentUserName()} @@ -315,10 +315,10 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase | |""".stripMargin) assert(result.collect()(2).length == 4) - assert(result.collect()(2)(1) === "tbl2") + assert(result.collect()(2)(1) === table2) assert(result.collect()(2)(3) === s"""Namespace: ns1.ns2 - |Table: tbl2 + |Table: $table2 |Type: MANAGED |Provider: _ |Owner: ${Utils.getCurrentUserName()} From 892de5351cb3dec318a2298308a969588c646acd Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 19 Oct 2023 09:22:41 +0800 Subject: [PATCH 13/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../sql/execution/command/v2/ShowTablesSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index bf399afba1be..3834d845802f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -253,7 +253,7 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase assert(result7.collect()(0)(1) === "tbl") assert(result7.collect()(0)(3) === s"""Namespace: $namespace - |Table: tbl + |Table: $table |Type: MANAGED |Location: s3://bucket/path |Provider: _ @@ -287,8 +287,8 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase assert(result.collect()(0).length == 4) assert(result.collect()(0)(1) === "tbl") assert(result.collect()(0)(3) === - s"""Namespace: ns1.ns2 - |Table: tbl + s"""Namespace: $namespace + |Table: $table |Type: MANAGED |Provider: _ |Owner: ${Utils.getCurrentUserName()} @@ -302,7 +302,7 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase assert(result.collect()(1).length == 4) assert(result.collect()(1)(1) === table1) assert(result.collect()(1)(3) === - s"""Namespace: ns1.ns2 + s"""Namespace: $namespace |Table: $table1 |Type: MANAGED |Provider: _ @@ -317,7 +317,7 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase assert(result.collect()(2).length == 4) assert(result.collect()(2)(1) === table2) assert(result.collect()(2)(3) === - s"""Namespace: ns1.ns2 + s"""Namespace: $namespace |Table: $table2 |Type: MANAGED |Provider: _ From d8989320de95685f88fb65d8dc8be8a7144e9b0d Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sun, 22 Oct 2023 20:17:37 +0800 Subject: [PATCH 14/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../sql/errors/QueryCompilationErrors.scala | 4 +- .../v2/DataSourceV2Implicits.scala | 6 +- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../datasources/v2/ShowTablesExec.scala | 29 +- .../command/ShowTablesSuiteBase.scala | 511 +++++++++++++++++- .../command/v1/ShowTablesSuite.scala | 11 - .../command/v2/ShowTablesSuite.scala | 287 +--------- 7 files changed, 530 insertions(+), 320 deletions(-) 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 7381e83ca4b9..3a7d55e79370 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 @@ -2464,7 +2464,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat errorClass = "_LEGACY_ERROR_TEMP_1231", messageParameters = Map( "key" -> key, - "tblName" -> tblName)) + "tblName" -> toSQLId(tblName))) } def invalidPartitionSpecError( @@ -2476,7 +2476,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map( "specKeys" -> specKeys, "partitionColumnNames" -> partitionColumnNames.mkString(", "), - "tableName" -> tableName)) + "tableName" -> toSQLId(tableName))) } def columnAlreadyExistsError(columnName: String): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index c68bc830e8e8..d29275f27dca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -30,10 +30,6 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Implicits { implicit class TableHelper(table: Table) { - def isReadable: Boolean = table match { - case _: SupportsRead => true - case _ => false - } def asReadable: SupportsRead = { table match { @@ -70,7 +66,7 @@ object DataSourceV2Implicits { } } - def isPartitionable: Boolean = table match { + def supportsPartitions: Boolean = table match { case _: SupportsPartitionManagement => true case _ => false } 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 db1bc5257c7a..823d4531a2ff 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 @@ -409,7 +409,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat pattern, partitionSpec @ (None | Some(_: ResolvedPartitionSpec)), output) => - ShowTablesExec(output, catalog.asTableCatalog, ns, Some(pattern), true, + ShowTablesExec(output, catalog.asTableCatalog, ns, Some(pattern), isExtended = true, partitionSpec.map(_.asInstanceOf[ResolvedPartitionSpec])) :: Nil case SetCatalogAndNamespace(ResolvedNamespace(catalog, ns)) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 5b0be0a62d3f..8232585f5a58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.{StringType, StructType} /** * Physical plan node for showing tables. @@ -51,14 +51,13 @@ case class ShowTablesExec( // Show the information of tables. val identifiers = catalog.listTables(namespace.toArray) identifiers.map { identifier => - if (pattern.map(StringUtils.filterPattern( - Seq(identifier.name()), _).nonEmpty).getOrElse(true)) { + if (pattern.forall(StringUtils.filterPattern(Seq(identifier.name()), _).nonEmpty)) { val isTemp = isTempView(identifier) if (isExtended) { val table = catalog.loadTable(identifier) - val information = extendedTable(identifier, table) - rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), isTemp, - s"$information\n") + val information = extendedTable(identifier, table) + rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), isTemp, + s"$information\n") } else { rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), isTemp) } @@ -70,9 +69,8 @@ case class ShowTablesExec( val table = catalog.loadTable(identifier) val isTemp = isTempView(identifier) val information = extendedPartition(identifier, table.asPartitionable, partitionSpec.get) - rows += toCatalystRow(namespace.quoted, table.name(), isTemp, s"$information\n") + rows += toCatalystRow(namespace.quoted, identifier.name(), isTemp, s"$information\n") } - rows.toSeq } @@ -116,14 +114,17 @@ case class ShowTablesExec( } // Partition Provider & Partition Columns - if (table.isPartitionable && !table.asPartitionable.partitionSchema().isEmpty) { + var partitionColumns = new StructType() + if (table.supportsPartitions && table.asPartitionable.partitionSchema().nonEmpty) { + partitionColumns = table.asPartitionable.partitionSchema() results.put("Partition Provider", "Catalog") results.put("Partition Columns", table.asPartitionable.partitionSchema().map( - field => quoteIdentifier(field.name)).mkString(", ")) + field => quoteIdentifier(field.name)).mkString("[", ", ", "]")) } - if (table.isReadable) { - if (table.schema().nonEmpty) results.put("Schema", table.schema().treeString) + if (table.schema().nonEmpty) { + val dataColumns = table.schema().filterNot(partitionColumns.contains) + results.put("Schema", StructType(dataColumns ++ partitionColumns).treeString) } results.map { case (key, value) => @@ -141,7 +142,7 @@ case class ShowTablesExec( val partitionSchema = partitionTable.partitionSchema() val (names, ident) = (resolvedPartitionSpec.names, resolvedPartitionSpec.ident) val partitionIdentifiers = partitionTable.listPartitionIdentifiers(names.toArray, ident) - if (partitionIdentifiers.length == 0) { + if (partitionIdentifiers.isEmpty) { throw QueryExecutionErrors.notExistPartitionError(identifier.toString, ident, partitionSchema) } val row = partitionIdentifiers.head @@ -171,6 +172,6 @@ case class ShowTablesExec( results.map { case (key, value) => if (value.isEmpty) key else s"$key: $value" - }.mkString("", "\n", "\n") + }.mkString("", "\n", "") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index 5f56b91db8f9..5245154e5304 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Utils /** * This base suite contains unified tests for the `SHOW TABLES` command that check V1 and V2 @@ -126,4 +127,512 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } } + + test("show table in a not existing namespace") { + checkError( + exception = intercept[AnalysisException] { + sql(s"SHOW TABLES IN $catalog.nonexist") + }, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`nonexist`")) + } + + test("show table extended in a not existing namespace") { + checkError( + exception = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.nonexist LIKE '*tbl*'") + }, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`nonexist`")) + } + + test("show table extended in a not existing table") { + val namespace = "ns1" + val table = "nonexist" + withNamespaceAndTable(namespace, table, catalog) { _ => + val result = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '*$table*'") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect().isEmpty) + } + } + + test("show table extended in non-partitioned table") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") + val e = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") + } + checkError( + exception = e, + errorClass = e.getErrorClass, + parameters = e.getErrorClass match { + case "_LEGACY_ERROR_TEMP_1251" => + Map("action" -> "SHOW TABLE EXTENDED", "tableName" -> table) // v1 v2 + case "_LEGACY_ERROR_TEMP_1231" => + Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`") // hive + } + ) + } + } + + test("show table extended in a not existing partition") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") + checkError( + exception = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 2)") + }, + errorClass = "PARTITIONS_NOT_FOUND", + parameters = Map( + "partitionList" -> "PARTITION (`id` = 2)", + "tableName" -> "`ns1`.`tbl`" + ) + ) + } + } + + test("show table extended in multi partition key table") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id1, id2)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(0) === namespace) + assert(result.collect()(0)(1) === table) + assert(result.collect()(0)(2) === false) + val actualResult = exclude(result.collect()(0)(3).toString) + val expectedResult_v1_v2 = "Partition Values: [id1=1, id2=2]" + val expectedResult_hive = + """Partition Values: [id1=1, id2=2] + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1]""".stripMargin + assert(actualResult === expectedResult_v1_v2 || actualResult === expectedResult_hive) + + checkError( + exception = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + + s"LIKE '$table' PARTITION(id1 = 1)") + }, + errorClass = "_LEGACY_ERROR_TEMP_1232", + parameters = Map( + "specKeys" -> "id1", + "partitionColumnNames" -> "id1, id2", + "tableName" -> s"`$catalog`.`$namespace`.`$table`") + ) + } + } + + test("show table extended in multi tables") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { _ => + sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id)") + val table1 = "tbl1" + val table2 = "tbl2" + withTable(table1, table2) { + sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + + s"$defaultUsing PARTITIONED BY (id1)") + sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + + s"$defaultUsing PARTITIONED BY (id2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") + .sort("tableName") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect().length == 3) + + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(1) === table) + assert(result.collect()(0)(2) === false) + val actualResult_0_3 = exclude(result.collect()(0)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_0_3_v1 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table + |Type: MANAGED + |Provider: parquet + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + val expectedResult_0_3_v2 = + s"""Namespace: $namespace + |Table: $table + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + + // exclude "Table Properties" + val expectedResult_0_3_hive = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table + |Owner: ${Utils.getCurrentUserName()} + |Type: MANAGED + |Provider: hive + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + assert(actualResult_0_3 === expectedResult_0_3_v1 || + actualResult_0_3 === expectedResult_0_3_v2 || + actualResult_0_3 === expectedResult_0_3_hive) + + assert(result.collect()(1).length == 4) + assert(result.collect()(1)(1) === table1) + assert(result.collect()(1)(2) === false) + val actualResult_1_3 = exclude(result.collect()(1)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_1_3_v1 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table1 + |Type: MANAGED + |Provider: parquet + |Partition Provider: Catalog + |Partition Columns: [`id1`] + |Schema: root + | |-- data1: string (nullable = true) + | |-- id1: long (nullable = true)""".stripMargin + val expectedResult_1_3_v2 = + s"""Namespace: $namespace + |Table: $table1 + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: [`id1`] + |Schema: root + | |-- data1: string (nullable = true) + | |-- id1: long (nullable = true)""".stripMargin + + // exclude "Table Properties" + val expectedResult_1_3_hive = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table1 + |Owner: ${Utils.getCurrentUserName()} + |Type: MANAGED + |Provider: hive + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Provider: Catalog + |Partition Columns: [`id1`] + |Schema: root + | |-- data1: string (nullable = true) + | |-- id1: long (nullable = true)""".stripMargin + assert(actualResult_1_3 === expectedResult_1_3_v1 || + actualResult_1_3 === expectedResult_1_3_v2 || + actualResult_1_3 === expectedResult_1_3_hive) + + assert(result.collect()(2).length == 4) + assert(result.collect()(2)(1) === table2) + assert(result.collect()(2)(2) === false) + val actualResult_2_3 = exclude(result.collect()(2)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_2_3_v1 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table2 + |Type: MANAGED + |Provider: parquet + |Partition Provider: Catalog + |Partition Columns: [`id2`] + |Schema: root + | |-- data2: string (nullable = true) + | |-- id2: long (nullable = true)""".stripMargin + val expectedResult_2_3_v2 = + s"""Namespace: $namespace + |Table: $table2 + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: [`id2`] + |Schema: root + | |-- data2: string (nullable = true) + | |-- id2: long (nullable = true)""".stripMargin + + // exclude "Table Properties" + val expectedResult_2_3_hive = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table2 + |Owner: ${Utils.getCurrentUserName()} + |Type: MANAGED + |Provider: hive + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Provider: Catalog + |Partition Columns: [`id2`] + |Schema: root + | |-- data2: string (nullable = true) + | |-- id2: long (nullable = true)""".stripMargin + assert(actualResult_2_3 === expectedResult_2_3_v1 || + actualResult_2_3 === expectedResult_2_3_v2 || + actualResult_2_3 === expectedResult_2_3_hive) + } + } + } + + test("show table extended in/from") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { _ => + sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id)") + sql(s"ALTER TABLE $catalog.$namespace.$table ADD PARTITION (id = 1)") + + val result1 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table'") + assert(result1.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result1.collect()(0).length == 4) + assert(result1.collect()(0)(2) === false) + val actualResult1 = exclude(result1.collect()(0)(3).toString) + val expectedResult_v1 = + s"""Namespace: $namespace + |Table: $table + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + val expectedResult_v2 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table + |Type: MANAGED + |Provider: parquet + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + val expectedResult_hive = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table + |Owner: ${Utils.getCurrentUserName()} + |Type: MANAGED + |Provider: hive + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + assert(actualResult1 === expectedResult_v1 || + actualResult1 === expectedResult_v2 || + actualResult1 === expectedResult_hive) + + val result2 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table'") + assert(result2.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result2.collect()(0).length == 4) + assert(result2.collect()(0)(2) === false) + val actualResult2 = exclude(result2.collect()(0)(3).toString) + assert(actualResult2 === expectedResult_v1 || + actualResult2 === expectedResult_v2 || + actualResult2 === expectedResult_hive) + + val result3 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE 'tb*'") + assert(result3.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result3.collect()(0).length == 4) + assert(result3.collect()(0)(2) === false) + val actualResult3 = exclude(result3.collect()(0)(3).toString) + assert(actualResult3 === expectedResult_v1 || + actualResult3 === expectedResult_v2 || + actualResult3 === expectedResult_hive) + + val result4 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") + assert(result4.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result4.collect()(0).length == 4) + assert(result4.collect()(0)(2) === false) + val actualResult4 = exclude(result4.collect()(0)(3).toString) + assert(actualResult4 === expectedResult_v1 || + actualResult4 === expectedResult_v2 || + actualResult4 === expectedResult_hive) + + val result5 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE '$table' PARTITION(id = 1)") + assert(result5.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result5.collect()(0).length == 4) + assert(result5.collect()(0)(2) === false) + val actualResult5 = exclude(result5.collect()(0)(3).toString) + val expectedResultPartition_v1_v2 = "Partition Values: [id=1]" + val expectedResultPartition_hive = + """Partition Values: [id=1] + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1]""".stripMargin + assert(actualResult5 === expectedResultPartition_v1_v2 || + actualResult5 === expectedResultPartition_hive) + + val result6 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + + s"LIKE '$table' PARTITION(id = 1)") + assert(result6.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result6.collect()(0).length == 4) + assert(result6.collect()(0)(2) === false) + val actualResult6 = exclude(result6.collect()(0)(3).toString) + assert(actualResult6 === expectedResultPartition_v1_v2 || + actualResult6 === expectedResultPartition_hive) + + withTempDir { dir => + sql(s"ALTER TABLE $catalog.$namespace.$table " + + s"SET LOCATION 'file://${dir.getCanonicalPath}'") + val result7 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") + assert(result7.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result7.collect()(0).length == 4) + assert(result7.collect()(0)(1) === table) + assert(result7.collect()(0)(2) === false) + val actualResult7 = exclude(result7.collect()(0)(3).toString) + val expectedResult7_v1 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table + |Type: MANAGED + |Provider: parquet + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + val expectedResult7_v2 = + s"""Namespace: $namespace + |Table: $table + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Schema: root + | |-- id: long (nullable = true) + | |-- data: string (nullable = true)""".stripMargin + val expectedResult7_hive = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table + |Owner: ${Utils.getCurrentUserName()} + |Type: MANAGED + |Provider: hive + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + assert(actualResult7 === expectedResult7_v1 || + actualResult7 === expectedResult7_v2 || + actualResult7 === expectedResult7_hive) + } + } + } + + test("show table extended from temp view") { + val namespace = "ns" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + val viewName = "temp_view" + withView(viewName) { + sql(s"CREATE TEMPORARY VIEW $viewName AS SELECT id FROM $t") + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$viewName'") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + if (!result.collect().isEmpty) { + assert(result.collect()(0)(1) === viewName) + assert(result.collect()(0)(2) === true) + val actualResult = exclude(result.collect()(0)(3).toString) + val expectedResult_v1_hive = + s"""Table: $viewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.default + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult === expectedResult_v1_hive) + } + } + } + } + + // Exclude some non-deterministic values for easy comparison of results, + // such as `Created Time`, etc + private def exclude(text: String): String = { + text.split("\n").filter(line => + !line.startsWith("Created Time:") && + !line.startsWith("Last Access:") && + !line.startsWith("Created By:") && + !line.startsWith("Location:") && + !line.startsWith("Table Properties:") && + !line.startsWith("Partition Parameters:")).mkString("\n") + } + + test("show tables from temp view") { + val namespace = "ns" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + val viewName = "temp_view" + withView(viewName) { + sql(s"CREATE TEMPORARY VIEW $viewName AS SELECT id FROM $t") + val result = sql(s"SHOW TABLES FROM $catalog.$namespace LIKE '$viewName'") + assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary")) + if (!result.collect().isEmpty) { + assert(result.collect()(0)(1) === viewName) + val expectedResult_v1_hive = true + assert(result.collect()(0)(2) === expectedResult_v1_hive) + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index 5bda7d002dc5..c9cfc1fa3882 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.command.v1 import org.apache.spark.sql.{AnalysisException, Row, SaveMode} -import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException import org.apache.spark.sql.execution.command import org.apache.spark.sql.internal.SQLConf @@ -134,16 +133,6 @@ trait ShowTablesSuiteBase extends command.ShowTablesSuiteBase with command.Tests } } } - - test("show table in a not existing namespace") { - val e = intercept[NoSuchDatabaseException] { - runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq()) - } - checkError(e, - errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`unknown`")) - } - } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index 3834d845802f..c51b035e86cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -17,10 +17,8 @@ package org.apache.spark.sql.execution.command.v2 -import org.apache.spark.sql.{AnalysisException, Row} -import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchPartitionException} +import org.apache.spark.sql.Row import org.apache.spark.sql.execution.command -import org.apache.spark.util.Utils /** * The class contains tests for the `SHOW TABLES` command to check V2 table catalogs. @@ -49,287 +47,4 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(Row("", "table", false))) } } - - test("show table in a not existing namespace") { - val e = intercept[NoSuchNamespaceException] { - runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq()) - } - checkError(e, - errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`unknown`")) - } - - test("show table extended in a not existing namespace") { - checkError( - exception = intercept[NoSuchNamespaceException] { - sql(s"SHOW TABLE EXTENDED FROM $catalog.unknown LIKE '*tbl*'") - }, - errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`unknown`")) - } - - test("show table extended in a not existing table") { - val table = "nonexist" - withTable(s"$catalog.$table") { - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog LIKE '*$table*'") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect().isEmpty) - } - } - - test("show table extended in non-partitioned table") { - val namespace = "ns1.ns2" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") - checkError( - exception = intercept[AnalysisException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") - }, - errorClass = "_LEGACY_ERROR_TEMP_1231", - parameters = Map("key" -> "id", "tblName" -> s"$catalog.$namespace.$table") - ) - } - } - - test("show table extended in multi-partition table") { - val namespace = "ns1.ns2" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id1, id2)") - sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(0) === namespace) - assert(result.collect()(0)(1) === s"$catalog.$namespace.$table") - assert(result.collect()(0)(2) === false) - assert(result.collect()(0)(3) === - """Partition Values: [id1=1, id2=2] - | - |""".stripMargin) - - checkError( - exception = intercept[AnalysisException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)") - }, - errorClass = "_LEGACY_ERROR_TEMP_1232", - parameters = Map( - "specKeys" -> "id1", - "partitionColumnNames" -> "id1, id2", - "tableName" -> s"$catalog.$namespace.$table") - ) - } - } - - test("show table extended in a not existing partition") { - val namespace = "ns1.ns2" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") - sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") - checkError( - exception = intercept[NoSuchPartitionException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 2)") - }, - errorClass = "PARTITIONS_NOT_FOUND", - parameters = Map( - "partitionList" -> "PARTITION (`id` = 2)", - "tableName" -> "`ns1`.`ns2`.`tbl`" - ) - ) - } - } - - test("show table extended for v2 tables") { - val namespace = "ns1.ns2" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") - sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") - - val result1 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table'") - assert(result1.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result1.collect()(0).length == 4) - assert(result1.collect()(0)(3) == - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: `id` - |Schema: root - | |-- id: long (nullable = true) - | |-- data: string (nullable = true) - | - |""".stripMargin) - - val result2 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table'") - assert(result2.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result2.collect()(0).length == 4) - assert(result2.collect()(0)(3) == - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: `id` - |Schema: root - | |-- id: long (nullable = true) - | |-- data: string (nullable = true) - | - |""".stripMargin) - - val result3 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE 'tb*'") - assert(result3.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result3.collect()(0).length == 4) - assert(result3.collect()(0)(3) == - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: `id` - |Schema: root - | |-- id: long (nullable = true) - | |-- data: string (nullable = true) - | - |""".stripMargin) - - val result4 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") - assert(result4.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result4.collect()(0).length == 4) - assert(result4.collect()(0)(3) === - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: `id` - |Schema: root - | |-- id: long (nullable = true) - | |-- data: string (nullable = true) - | - |""".stripMargin) - - val result5 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - s"LIKE '$table' PARTITION(id = 1)") - assert(result5.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result5.collect()(0).length == 4) - assert(result5.collect()(0)(3) === - """Partition Values: [id=1] - | - |""".stripMargin) - - val result6 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + - s"LIKE '$table' PARTITION(id = 1)") - assert(result6.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result6.collect()(0).length == 4) - assert(result6.collect()(0)(3) === - """Partition Values: [id=1] - | - |""".stripMargin) - - sql(s"ALTER TABLE $tbl SET LOCATION 's3://bucket/path'") - val result7 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") - assert(result7.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result7.collect()(0).length == 4) - assert(result7.collect()(0)(1) === "tbl") - assert(result7.collect()(0)(3) === - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Location: s3://bucket/path - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Schema: root - | |-- id: long (nullable = true) - | |-- data: string (nullable = true) - | - |""".stripMargin) - } - } - - test("show table extended for v2 multi tables") { - val namespace = "ns1.ns2" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") - val table1 = "tbl1" - val table2 = "tbl2" - withTable(table1, table2) { - sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + - s"$defaultUsing PARTITIONED BY (id1)") - sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + - s"$defaultUsing PARTITIONED BY (id2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") - .sort("tableName") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect().length == 3) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(1) === "tbl") - assert(result.collect()(0)(3) === - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: `id` - |Schema: root - | |-- id: long (nullable = true) - | |-- data: string (nullable = true) - | - |""".stripMargin) - assert(result.collect()(1).length == 4) - assert(result.collect()(1)(1) === table1) - assert(result.collect()(1)(3) === - s"""Namespace: $namespace - |Table: $table1 - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: `id1` - |Schema: root - | |-- id1: long (nullable = true) - | |-- data1: string (nullable = true) - | - |""".stripMargin) - assert(result.collect()(2).length == 4) - assert(result.collect()(2)(1) === table2) - assert(result.collect()(2)(3) === - s"""Namespace: $namespace - |Table: $table2 - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: `id2` - |Schema: root - | |-- id2: long (nullable = true) - | |-- data2: string (nullable = true) - | - |""".stripMargin) - } - } - } } From 3b12d76d2151bfa6cc732c345436cb838bb119b3 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 27 Oct 2023 19:39:22 +0800 Subject: [PATCH 15/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../catalyst/analysis/ResolveCatalogs.scala | 2 +- .../analysis/ResolvePartitionSpec.scala | 14 +- .../sql/catalyst/parser/AstBuilder.scala | 25 +- .../catalyst/plans/logical/v2Commands.scala | 32 +- .../analysis/ResolveSessionCatalog.scala | 22 +- .../datasources/v2/DataSourceV2Strategy.scala | 11 +- .../datasources/v2/ShowTablesExec.scala | 9 +- .../command/ShowTablesParserSuite.scala | 34 +- .../command/ShowTablesSuiteBase.scala | 432 +----------------- .../command/v1/ShowTablesSuite.scala | 120 +++++ .../command/v2/ShowTablesSuite.scala | 125 ++++- .../execution/command/ShowTablesSuite.scala | 141 ++++++ 12 files changed, 483 insertions(+), 484 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 788f79cde99e..826c5b524088 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -48,7 +48,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) } case s @ ShowTables(UnresolvedNamespace(Seq()), _, _) => s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) - case s @ ShowTableExtended(UnresolvedNamespace(Seq()), _, _, _) => + case s @ ShowTablesExtended(UnresolvedNamespace(Seq()), _, _) => s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) case s @ ShowViews(UnresolvedNamespace(Seq()), _, _) => s.copy(namespace = ResolvedNamespace(currentCatalog, catalogManager.currentNamespace)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala index 23e1fefc7b87..90a502653d04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala @@ -20,11 +20,11 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ShowTableExtended, V2PartitionCommand} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2PartitionCommand} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND import org.apache.spark.sql.catalyst.util.CharVarcharUtils -import org.apache.spark.sql.connector.catalog.{Identifier, SupportsPartitionManagement, TableCatalog} +import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.PartitioningUtils.{castPartitionSpec, normalizePartitionSpec, requireExactMatchedPartitionSpec} @@ -50,16 +50,6 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] { } case _ => command } - case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace), - pattern, _, _) if command.childrenResolved && !command.resolved => - val table = catalog.loadTable(Identifier.of(namespace.toArray, pattern)) - table match { - case s: SupportsPartitionManagement => command.transformExpressions { - case partSpecs: UnresolvedPartitionSpec => - resolvePartitionSpec(table.name, partSpecs, s.partitionSchema(), false) - } - case _ => command - } } private def resolvePartitionSpec( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 2b5f542f22bb..ce92f09db82f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -4055,19 +4055,28 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { } /** - * Create a [[ShowTableExtended]] command. + * Create a [[ShowTablesExtended]] or [[ShowTablePartition]] command. */ override def visitShowTableExtended( ctx: ShowTableExtendedContext): LogicalPlan = withOrigin(ctx) { - val partitionKeys = Option(ctx.partitionSpec).map { specCtx => - UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(specCtx), None) + def createUnresolvedTable( + nsCtx: IdentifierReferenceContext, + patternCtx: StringLitContext): LogicalPlan = { + val ns = Option(nsCtx).map(x => visitMultipartIdentifier(x.multipartIdentifier())) + UnresolvedTable(ns.getOrElse(Seq.empty[String]) :+ string(visitStringLit(patternCtx)), + "SHOW TABLE EXTENDED ... PARTITION ...") } - val ns = if (ctx.identifierReference() != null) { - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) - } else { - UnresolvedNamespace(Seq.empty[String]) + Option(ctx.partitionSpec).map { spec => + val table = createUnresolvedTable(ctx.identifierReference(), ctx.pattern) + ShowTablePartition(table, UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(spec))) + }.getOrElse { + val ns = if (ctx.identifierReference() != null) { + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) + } else { + UnresolvedNamespace(Seq.empty[String]) + } + ShowTablesExtended(ns, string(visitStringLit(ctx.pattern))) } - ShowTableExtended(ns, string(visitStringLit(ctx.pattern)), partitionKeys) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index ac4098d4e410..6bb175fc6de3 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 @@ -884,19 +884,39 @@ object ShowTables { } /** - * The logical plan of the SHOW TABLE EXTENDED command. + * The logical plan of the SHOW TABLE EXTENDED (without PARTITION) command. */ -case class ShowTableExtended( +case class ShowTablesExtended( namespace: LogicalPlan, pattern: String, - partitionSpec: Option[PartitionSpec], - override val output: Seq[Attribute] = ShowTableExtended.getOutputAttrs) extends UnaryCommand { + override val output: Seq[Attribute] = ShowTablesExtended.getOutputAttrs) + extends UnaryCommand { override def child: LogicalPlan = namespace - override protected def withNewChildInternal(newChild: LogicalPlan): ShowTableExtended = + override protected def withNewChildInternal(newChild: LogicalPlan): ShowTablesExtended = copy(namespace = newChild) } -object ShowTableExtended { +object ShowTablesExtended { + def getOutputAttrs: Seq[Attribute] = Seq( + AttributeReference("namespace", StringType, nullable = false)(), + AttributeReference("tableName", StringType, nullable = false)(), + AttributeReference("isTemporary", BooleanType, nullable = false)(), + AttributeReference("information", StringType, nullable = false)()) +} + +/** + * The logical plan of the SHOW TABLE EXTENDED ... PARTITION ... command. + */ +case class ShowTablePartition( + table: LogicalPlan, + partitionSpec: PartitionSpec, + override val output: Seq[Attribute] = ShowTablePartition.getOutputAttrs) + extends V2PartitionCommand { + override protected def withNewChildInternal(newChild: LogicalPlan): ShowTablePartition = + copy(table = newChild) +} + +object ShowTablePartition { def getOutputAttrs: Seq[Attribute] = Seq( AttributeReference("namespace", StringType, nullable = false)(), AttributeReference("tableName", StringType, nullable = false)(), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 56515ca7e37a..e2cf12c1bd74 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 @@ -240,10 +240,9 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case ShowTables(DatabaseInSessionCatalog(db), pattern, output) if conf.useV1Command => ShowTablesCommand(Some(db), pattern, output) - case ShowTableExtended( + case ShowTablesExtended( DatabaseInSessionCatalog(db), pattern, - partitionSpec @ (None | Some(UnresolvedPartitionSpec(_, _))), output) => val newOutput = if (conf.getConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA)) { assert(output.length == 4) @@ -251,8 +250,23 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) } else { output } - val tablePartitionSpec = partitionSpec.map(_.asInstanceOf[UnresolvedPartitionSpec].spec) - ShowTablesCommand(Some(db), Some(pattern), newOutput, true, tablePartitionSpec) + ShowTablesCommand(Some(db), Some(pattern), newOutput, isExtended = true) + + case ShowTablePartition( + ResolvedTable(catalog, _, table: V1Table, _), + partitionSpec, + output) if isSessionCatalog(catalog) => + val newOutput = if (conf.getConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA)) { + assert(output.length == 4) + output.head.withName("database") +: output.tail + } else { + output + } + val tablePartitionSpec = Option(partitionSpec).map( + _.asInstanceOf[UnresolvedPartitionSpec].spec) + ShowTablesCommand(table.catalogTable.identifier.database, + Some(table.catalogTable.identifier.table), newOutput, + isExtended = true, tablePartitionSpec) // ANALYZE TABLE works on permanent views if the views are cached. case AnalyzeTable(ResolvedV1TableOrViewIdentifier(ident), partitionSpec, noScan) => 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 823d4531a2ff..7952d2d38da1 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 @@ -404,13 +404,16 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case ShowTables(ResolvedNamespace(catalog, ns), pattern, output) => ShowTablesExec(output, catalog.asTableCatalog, ns, pattern) :: Nil - case ShowTableExtended( + case ShowTablesExtended( ResolvedNamespace(catalog, ns), pattern, - partitionSpec @ (None | Some(_: ResolvedPartitionSpec)), output) => - ShowTablesExec(output, catalog.asTableCatalog, ns, Some(pattern), isExtended = true, - partitionSpec.map(_.asInstanceOf[ResolvedPartitionSpec])) :: Nil + ShowTablesExec(output, catalog.asTableCatalog, ns, Some(pattern), + isExtended = true, partitionSpec = None) :: Nil + + case ShowTablePartition(r: ResolvedTable, part, output) => + ShowTablesExec(output, r.catalog, r.identifier.namespace(), Some(r.identifier.name()), + isExtended = true, partitionSpec = Some(Seq(part).asResolvedPartitionSpecs.head)) :: Nil case SetCatalogAndNamespace(ResolvedNamespace(catalog, ns)) => val catalogManager = session.sessionState.catalogManager diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 8232585f5a58..96076c301a91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -55,7 +55,7 @@ case class ShowTablesExec( val isTemp = isTempView(identifier) if (isExtended) { val table = catalog.loadTable(identifier) - val information = extendedTable(identifier, table) + val information = extendedTable(catalog.name, identifier, table) rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), isTemp, s"$information\n") } else { @@ -81,9 +81,14 @@ case class ShowTablesExec( } } - private def extendedTable(identifier: Identifier, table: Table): String = { + private def extendedTable( + catalogName: String, + identifier: Identifier, + table: Table): String = { val results = new mutable.LinkedHashMap[String, String]() + results.put("Catalog", catalogName) + if (!identifier.namespace().isEmpty) { results.put("Namespace", identifier.namespace().quoted) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala index d68e1233f7ab..d70853b6360f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesParserSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace, UnresolvedPartitionSpec} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace, UnresolvedPartitionSpec, UnresolvedTable} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan -import org.apache.spark.sql.catalyst.plans.logical.{ShowTableExtended, ShowTables} +import org.apache.spark.sql.catalyst.plans.logical.{ShowTablePartition, ShowTables, ShowTablesExtended} import org.apache.spark.sql.test.SharedSparkSession class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession { @@ -52,32 +52,32 @@ class ShowTablesParserSuite extends AnalysisTest with SharedSparkSession { test("show table extended") { comparePlans( parsePlan("SHOW TABLE EXTENDED LIKE '*test*'"), - ShowTableExtended(UnresolvedNamespace(Seq.empty[String]), "*test*", None)) + ShowTablesExtended(UnresolvedNamespace(Seq.empty[String]), "*test*")) comparePlans( parsePlan(s"SHOW TABLE EXTENDED FROM $catalog.ns1.ns2 LIKE '*test*'"), - ShowTableExtended(UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*", None)) + ShowTablesExtended(UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*")) comparePlans( parsePlan(s"SHOW TABLE EXTENDED IN $catalog.ns1.ns2 LIKE '*test*'"), - ShowTableExtended(UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*", None)) + ShowTablesExtended(UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), "*test*")) + comparePlans( parsePlan("SHOW TABLE EXTENDED LIKE '*test*' PARTITION(ds='2008-04-09', hr=11)"), - ShowTableExtended( - UnresolvedNamespace(Seq.empty[String]), - "*test*", - Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09", "hr" -> "11"))))) + ShowTablePartition( + UnresolvedTable(Seq("*test*"), "SHOW TABLE EXTENDED ... PARTITION ..."), + UnresolvedPartitionSpec(Map("ds" -> "2008-04-09", "hr" -> "11")))) comparePlans( parsePlan(s"SHOW TABLE EXTENDED FROM $catalog.ns1.ns2 LIKE '*test*' " + "PARTITION(ds='2008-04-09')"), - ShowTableExtended( - UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), - "*test*", - Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09"))))) + ShowTablePartition( + UnresolvedTable(Seq(catalog, "ns1", "ns2", "*test*"), + "SHOW TABLE EXTENDED ... PARTITION ..."), + UnresolvedPartitionSpec(Map("ds" -> "2008-04-09")))) comparePlans( parsePlan(s"SHOW TABLE EXTENDED IN $catalog.ns1.ns2 LIKE '*test*' " + "PARTITION(ds='2008-04-09')"), - ShowTableExtended( - UnresolvedNamespace(Seq(catalog, "ns1", "ns2")), - "*test*", - Some(UnresolvedPartitionSpec(Map("ds" -> "2008-04-09"))))) + ShowTablePartition( + UnresolvedTable(Seq(catalog, "ns1", "ns2", "*test*"), + "SHOW TABLE EXTENDED ... PARTITION ..."), + UnresolvedPartitionSpec(Map("ds" -> "2008-04-09")))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index 5245154e5304..c40475bd77e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.util.Utils /** * This base suite contains unified tests for the `SHOW TABLES` command that check V1 and V2 @@ -157,27 +156,6 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } - test("show table extended in non-partitioned table") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") - val e = intercept[AnalysisException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") - } - checkError( - exception = e, - errorClass = e.getErrorClass, - parameters = e.getErrorClass match { - case "_LEGACY_ERROR_TEMP_1251" => - Map("action" -> "SHOW TABLE EXTENDED", "tableName" -> table) // v1 v2 - case "_LEGACY_ERROR_TEMP_1231" => - Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`") // hive - } - ) - } - } - test("show table extended in a not existing partition") { val namespace = "ns1" val table = "tbl" @@ -197,7 +175,8 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } - test("show table extended in multi partition key table") { + test("show table extended in multi partition key - " + + "the command's partition parameters are incomplete") { val namespace = "ns1" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { tbl => @@ -205,24 +184,6 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { s"$defaultUsing PARTITIONED BY (id1, id2)") sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(0) === namespace) - assert(result.collect()(0)(1) === table) - assert(result.collect()(0)(2) === false) - val actualResult = exclude(result.collect()(0)(3).toString) - val expectedResult_v1_v2 = "Partition Values: [id1=1, id2=2]" - val expectedResult_hive = - """Partition Values: [id1=1, id2=2] - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1]""".stripMargin - assert(actualResult === expectedResult_v1_v2 || actualResult === expectedResult_hive) - checkError( exception = intercept[AnalysisException] { sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + @@ -237,377 +198,9 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } - test("show table extended in multi tables") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { _ => - sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id)") - val table1 = "tbl1" - val table2 = "tbl2" - withTable(table1, table2) { - sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + - s"$defaultUsing PARTITIONED BY (id1)") - sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + - s"$defaultUsing PARTITIONED BY (id2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") - .sort("tableName") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect().length == 3) - - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(1) === table) - assert(result.collect()(0)(2) === false) - val actualResult_0_3 = exclude(result.collect()(0)(3).toString) - - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_0_3_v1 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table - |Type: MANAGED - |Provider: parquet - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - val expectedResult_0_3_v2 = - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - - // exclude "Table Properties" - val expectedResult_0_3_hive = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table - |Owner: ${Utils.getCurrentUserName()} - |Type: MANAGED - |Provider: hive - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1] - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - assert(actualResult_0_3 === expectedResult_0_3_v1 || - actualResult_0_3 === expectedResult_0_3_v2 || - actualResult_0_3 === expectedResult_0_3_hive) - - assert(result.collect()(1).length == 4) - assert(result.collect()(1)(1) === table1) - assert(result.collect()(1)(2) === false) - val actualResult_1_3 = exclude(result.collect()(1)(3).toString) - - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_1_3_v1 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table1 - |Type: MANAGED - |Provider: parquet - |Partition Provider: Catalog - |Partition Columns: [`id1`] - |Schema: root - | |-- data1: string (nullable = true) - | |-- id1: long (nullable = true)""".stripMargin - val expectedResult_1_3_v2 = - s"""Namespace: $namespace - |Table: $table1 - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: [`id1`] - |Schema: root - | |-- data1: string (nullable = true) - | |-- id1: long (nullable = true)""".stripMargin - - // exclude "Table Properties" - val expectedResult_1_3_hive = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table1 - |Owner: ${Utils.getCurrentUserName()} - |Type: MANAGED - |Provider: hive - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1] - |Partition Provider: Catalog - |Partition Columns: [`id1`] - |Schema: root - | |-- data1: string (nullable = true) - | |-- id1: long (nullable = true)""".stripMargin - assert(actualResult_1_3 === expectedResult_1_3_v1 || - actualResult_1_3 === expectedResult_1_3_v2 || - actualResult_1_3 === expectedResult_1_3_hive) - - assert(result.collect()(2).length == 4) - assert(result.collect()(2)(1) === table2) - assert(result.collect()(2)(2) === false) - val actualResult_2_3 = exclude(result.collect()(2)(3).toString) - - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_2_3_v1 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table2 - |Type: MANAGED - |Provider: parquet - |Partition Provider: Catalog - |Partition Columns: [`id2`] - |Schema: root - | |-- data2: string (nullable = true) - | |-- id2: long (nullable = true)""".stripMargin - val expectedResult_2_3_v2 = - s"""Namespace: $namespace - |Table: $table2 - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: [`id2`] - |Schema: root - | |-- data2: string (nullable = true) - | |-- id2: long (nullable = true)""".stripMargin - - // exclude "Table Properties" - val expectedResult_2_3_hive = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table2 - |Owner: ${Utils.getCurrentUserName()} - |Type: MANAGED - |Provider: hive - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1] - |Partition Provider: Catalog - |Partition Columns: [`id2`] - |Schema: root - | |-- data2: string (nullable = true) - | |-- id2: long (nullable = true)""".stripMargin - assert(actualResult_2_3 === expectedResult_2_3_v1 || - actualResult_2_3 === expectedResult_2_3_v2 || - actualResult_2_3 === expectedResult_2_3_hive) - } - } - } - - test("show table extended in/from") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { _ => - sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id)") - sql(s"ALTER TABLE $catalog.$namespace.$table ADD PARTITION (id = 1)") - - val result1 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table'") - assert(result1.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result1.collect()(0).length == 4) - assert(result1.collect()(0)(2) === false) - val actualResult1 = exclude(result1.collect()(0)(3).toString) - val expectedResult_v1 = - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - val expectedResult_v2 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table - |Type: MANAGED - |Provider: parquet - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - val expectedResult_hive = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table - |Owner: ${Utils.getCurrentUserName()} - |Type: MANAGED - |Provider: hive - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1] - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - assert(actualResult1 === expectedResult_v1 || - actualResult1 === expectedResult_v2 || - actualResult1 === expectedResult_hive) - - val result2 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table'") - assert(result2.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result2.collect()(0).length == 4) - assert(result2.collect()(0)(2) === false) - val actualResult2 = exclude(result2.collect()(0)(3).toString) - assert(actualResult2 === expectedResult_v1 || - actualResult2 === expectedResult_v2 || - actualResult2 === expectedResult_hive) - - val result3 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE 'tb*'") - assert(result3.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result3.collect()(0).length == 4) - assert(result3.collect()(0)(2) === false) - val actualResult3 = exclude(result3.collect()(0)(3).toString) - assert(actualResult3 === expectedResult_v1 || - actualResult3 === expectedResult_v2 || - actualResult3 === expectedResult_hive) - - val result4 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") - assert(result4.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result4.collect()(0).length == 4) - assert(result4.collect()(0)(2) === false) - val actualResult4 = exclude(result4.collect()(0)(3).toString) - assert(actualResult4 === expectedResult_v1 || - actualResult4 === expectedResult_v2 || - actualResult4 === expectedResult_hive) - - val result5 = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - s"LIKE '$table' PARTITION(id = 1)") - assert(result5.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result5.collect()(0).length == 4) - assert(result5.collect()(0)(2) === false) - val actualResult5 = exclude(result5.collect()(0)(3).toString) - val expectedResultPartition_v1_v2 = "Partition Values: [id=1]" - val expectedResultPartition_hive = - """Partition Values: [id=1] - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1]""".stripMargin - assert(actualResult5 === expectedResultPartition_v1_v2 || - actualResult5 === expectedResultPartition_hive) - - val result6 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace " + - s"LIKE '$table' PARTITION(id = 1)") - assert(result6.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result6.collect()(0).length == 4) - assert(result6.collect()(0)(2) === false) - val actualResult6 = exclude(result6.collect()(0)(3).toString) - assert(actualResult6 === expectedResultPartition_v1_v2 || - actualResult6 === expectedResultPartition_hive) - - withTempDir { dir => - sql(s"ALTER TABLE $catalog.$namespace.$table " + - s"SET LOCATION 'file://${dir.getCanonicalPath}'") - val result7 = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE 'tb*'") - assert(result7.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - assert(result7.collect()(0).length == 4) - assert(result7.collect()(0)(1) === table) - assert(result7.collect()(0)(2) === false) - val actualResult7 = exclude(result7.collect()(0)(3).toString) - val expectedResult7_v1 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table - |Type: MANAGED - |Provider: parquet - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - val expectedResult7_v2 = - s"""Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Schema: root - | |-- id: long (nullable = true) - | |-- data: string (nullable = true)""".stripMargin - val expectedResult7_hive = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table - |Owner: ${Utils.getCurrentUserName()} - |Type: MANAGED - |Provider: hive - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1] - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - assert(actualResult7 === expectedResult7_v1 || - actualResult7 === expectedResult7_v2 || - actualResult7 === expectedResult7_hive) - } - } - } - - test("show table extended from temp view") { - val namespace = "ns" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { t => - sql(s"CREATE TABLE $t (id int) $defaultUsing") - val viewName = "temp_view" - withView(viewName) { - sql(s"CREATE TEMPORARY VIEW $viewName AS SELECT id FROM $t") - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$viewName'") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - if (!result.collect().isEmpty) { - assert(result.collect()(0)(1) === viewName) - assert(result.collect()(0)(2) === true) - val actualResult = exclude(result.collect()(0)(3).toString) - val expectedResult_v1_hive = - s"""Table: $viewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: $catalog.default - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult === expectedResult_v1_hive) - } - } - } - } - // Exclude some non-deterministic values for easy comparison of results, // such as `Created Time`, etc - private def exclude(text: String): String = { + protected def exclude(text: String): String = { text.split("\n").filter(line => !line.startsWith("Created Time:") && !line.startsWith("Last Access:") && @@ -616,23 +209,4 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { !line.startsWith("Table Properties:") && !line.startsWith("Partition Parameters:")).mkString("\n") } - - test("show tables from temp view") { - val namespace = "ns" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { t => - sql(s"CREATE TABLE $t (id int) $defaultUsing") - val viewName = "temp_view" - withView(viewName) { - sql(s"CREATE TEMPORARY VIEW $viewName AS SELECT id FROM $t") - val result = sql(s"SHOW TABLES FROM $catalog.$namespace LIKE '$viewName'") - assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary")) - if (!result.collect().isEmpty) { - assert(result.collect()(0)(1) === viewName) - val expectedResult_v1_hive = true - assert(result.collect()(0)(2) === expectedResult_v1_hive) - } - } - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index c9cfc1fa3882..270cf4620760 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -154,4 +154,124 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { } } } + + test("show table extended in non-partitioned table") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") + val e = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") + } + checkError( + exception = e, + errorClass = "_LEGACY_ERROR_TEMP_1251", + parameters = Map("action" -> "SHOW TABLE EXTENDED", "tableName" -> table) + ) + } + } + + test("show table extended in multi partition key - " + + "the command's partition parameters are complete") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id1, id2)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(0) === namespace) + assert(result.collect()(0)(1) === table) + assert(result.collect()(0)(2) === false) + val actualResult = exclude(result.collect()(0)(3).toString) + val expectedResult = "Partition Values: [id1=1, id2=2]" + assert(actualResult === expectedResult) + } + } + + test("show table extended in multi tables") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { _ => + sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id)") + val table1 = "tbl1" + val table2 = "tbl2" + withTable(table1, table2) { + sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + + s"$defaultUsing PARTITIONED BY (id1)") + sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + + s"$defaultUsing PARTITIONED BY (id2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") + .sort("tableName") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect().length == 3) + + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(1) === table) + assert(result.collect()(0)(2) === false) + val actualResult_0_3 = exclude(result.collect()(0)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_0_3 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table + |Type: MANAGED + |Provider: parquet + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + + assert(actualResult_0_3 === expectedResult_0_3) + + assert(result.collect()(1).length == 4) + assert(result.collect()(1)(1) === table1) + assert(result.collect()(1)(2) === false) + val actualResult_1_3 = exclude(result.collect()(1)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_1_3 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table1 + |Type: MANAGED + |Provider: parquet + |Partition Provider: Catalog + |Partition Columns: [`id1`] + |Schema: root + | |-- data1: string (nullable = true) + | |-- id1: long (nullable = true)""".stripMargin + assert(actualResult_1_3 === expectedResult_1_3) + + assert(result.collect()(2).length == 4) + assert(result.collect()(2)(1) === table2) + assert(result.collect()(2)(2) === false) + val actualResult_2_3 = exclude(result.collect()(2)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_2_3 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table2 + |Type: MANAGED + |Provider: parquet + |Partition Provider: Catalog + |Partition Columns: [`id2`] + |Schema: root + | |-- data2: string (nullable = true) + | |-- id2: long (nullable = true)""".stripMargin + assert(actualResult_2_3 === expectedResult_2_3) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index c51b035e86cb..bd2478e6d324 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql.execution.command.v2 -import org.apache.spark.sql.Row +import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.execution.command +import org.apache.spark.util.Utils /** * The class contains tests for the `SHOW TABLES` command to check V2 table catalogs. @@ -47,4 +48,126 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase runShowTablesSql(s"SHOW TABLES FROM $catalog", Seq(Row("", "table", false))) } } + + test("show table extended in non-partitioned table") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") + val e = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") + } + checkError( + exception = e, + errorClass = "_LEGACY_ERROR_TEMP_1231", + parameters = Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`") + ) + } + } + + test("show table extended in multi partition key - " + + "the command's partition parameters are complete") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id1, id2)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(0) === namespace) + assert(result.collect()(0)(1) === table) + assert(result.collect()(0)(2) === false) + val actualResult = exclude(result.collect()(0)(3).toString) + val expectedResult = "Partition Values: [id1=1, id2=2]" + assert(actualResult === expectedResult) + } + } + + test("show table extended in multi tables") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { _ => + sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id)") + val table1 = "tbl1" + val table2 = "tbl2" + withTable(table1, table2) { + sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + + s"$defaultUsing PARTITIONED BY (id1)") + sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + + s"$defaultUsing PARTITIONED BY (id2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") + .sort("tableName") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect().length == 3) + + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(1) === table) + assert(result.collect()(0)(2) === false) + val actualResult_0_3 = exclude(result.collect()(0)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_0_3 = + s"""Catalog: $catalog + |Namespace: $namespace + |Table: $table + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + assert(actualResult_0_3 === expectedResult_0_3) + + assert(result.collect()(1).length == 4) + assert(result.collect()(1)(1) === table1) + assert(result.collect()(1)(2) === false) + val actualResult_1_3 = exclude(result.collect()(1)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_1_3 = + s"""Catalog: $catalog + |Namespace: $namespace + |Table: $table1 + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: [`id1`] + |Schema: root + | |-- data1: string (nullable = true) + | |-- id1: long (nullable = true)""".stripMargin + assert(actualResult_1_3 === expectedResult_1_3) + + assert(result.collect()(2).length == 4) + assert(result.collect()(2)(1) === table2) + assert(result.collect()(2)(2) === false) + val actualResult_2_3 = exclude(result.collect()(2)(3).toString) + + // exclude "Created Time", "Last Access", "Created By", "Location" + val expectedResult_2_3 = + s"""Catalog: $catalog + |Namespace: $namespace + |Table: $table2 + |Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()} + |Partition Provider: Catalog + |Partition Columns: [`id2`] + |Schema: root + | |-- data2: string (nullable = true) + | |-- id2: long (nullable = true)""".stripMargin + assert(actualResult_2_3 === expectedResult_2_3) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala index 653a157e7625..7bf2e31915a7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql.hive.execution.command +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.command.v1 +import org.apache.spark.util.Utils /** * The class contains tests for the `SHOW TABLES` command to check V1 Hive external table catalog. @@ -33,4 +35,143 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { } } } + + test("show table extended in non-partitioned table") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") + val e = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") + } + checkError( + exception = e, + errorClass = "_LEGACY_ERROR_TEMP_1231", + parameters = Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`") + ) + } + } + + test("show table extended in multi partition key - " + + "the command's partition parameters are complete") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id1, id2)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(0) === namespace) + assert(result.collect()(0)(1) === table) + assert(result.collect()(0)(2) === false) + val actualResult = exclude(result.collect()(0)(3).toString) + val expectedResult = + """Partition Values: [id1=1, id2=2] + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1]""".stripMargin + assert(actualResult === expectedResult) + } + } + + test("show table extended in multi tables") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { _ => + sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id)") + val table1 = "tbl1" + val table2 = "tbl2" + withTable(table1, table2) { + sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + + s"$defaultUsing PARTITIONED BY (id1)") + sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + + s"$defaultUsing PARTITIONED BY (id2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") + .sort("tableName") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + assert(result.collect().length == 3) + + assert(result.collect()(0).length == 4) + assert(result.collect()(0)(1) === table) + assert(result.collect()(0)(2) === false) + val actualResult_0_3 = exclude(result.collect()(0)(3).toString) + + // exclude "Table Properties" + val expectedResult_0_3 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table + |Owner: ${Utils.getCurrentUserName()} + |Type: MANAGED + |Provider: hive + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Provider: Catalog + |Partition Columns: [`id`] + |Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + assert(actualResult_0_3 === expectedResult_0_3) + + assert(result.collect()(1).length == 4) + assert(result.collect()(1)(1) === table1) + assert(result.collect()(1)(2) === false) + val actualResult_1_3 = exclude(result.collect()(1)(3).toString) + + // exclude "Table Properties" + val expectedResult_1_3 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table1 + |Owner: ${Utils.getCurrentUserName()} + |Type: MANAGED + |Provider: hive + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Provider: Catalog + |Partition Columns: [`id1`] + |Schema: root + | |-- data1: string (nullable = true) + | |-- id1: long (nullable = true)""".stripMargin + assert(actualResult_1_3 === expectedResult_1_3) + + assert(result.collect()(2).length == 4) + assert(result.collect()(2)(1) === table2) + assert(result.collect()(2)(2) === false) + val actualResult_2_3 = exclude(result.collect()(2)(3).toString) + + // exclude "Table Properties" + val expectedResult_2_3 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $table2 + |Owner: ${Utils.getCurrentUserName()} + |Type: MANAGED + |Provider: hive + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Provider: Catalog + |Partition Columns: [`id2`] + |Schema: root + | |-- data2: string (nullable = true) + | |-- id2: long (nullable = true)""".stripMargin + assert(actualResult_2_3 === expectedResult_2_3) + } + } + } } From 474b9635dbbe2e8471bb4a79241a4ee3b22bf656 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 30 Oct 2023 11:23:38 +0800 Subject: [PATCH 16/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../main/resources/error/error-classes.json | 5 - .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../catalyst/plans/logical/v2Commands.scala | 15 +- .../sql/errors/QueryCompilationErrors.scala | 9 +- .../sql/errors/QueryExecutionErrors.scala | 11 +- .../v2/DataSourceV2Implicits.scala | 1 - .../analysis/ResolveSessionCatalog.scala | 2 - .../datasources/v2/DataSourceV2Strategy.scala | 7 +- .../datasources/v2/ShowTablesExec.scala | 146 ++---------------- .../analyzer-results/show-tables.sql.out | 13 +- .../sql-tests/results/show-tables.sql.out | 13 +- .../command/v1/ShowTablesSuite.scala | 134 +++++++++++++--- .../command/v2/ShowTablesSuite.scala | 131 +++++++++++++--- .../apache/spark/sql/test/SQLTestUtils.scala | 11 ++ .../execution/command/ShowTablesSuite.scala | 137 +++++++++++++--- 15 files changed, 406 insertions(+), 231 deletions(-) diff --git a/common/utils/src/main/resources/error/error-classes.json b/common/utils/src/main/resources/error/error-classes.json index f9cc0a865216..b6e3bfe4e52b 100644 --- a/common/utils/src/main/resources/error/error-classes.json +++ b/common/utils/src/main/resources/error/error-classes.json @@ -4703,11 +4703,6 @@ "Invalid bound function ': there are arguments but parameters returned from 'inputTypes()'." ] }, - "_LEGACY_ERROR_TEMP_1200" : { - "message" : [ - " is not supported for v2 tables." - ] - }, "_LEGACY_ERROR_TEMP_1201" : { "message" : [ "Cannot resolve column name \"\" among ()." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index ce92f09db82f..5d3979c947ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -4061,7 +4061,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { ctx: ShowTableExtendedContext): LogicalPlan = withOrigin(ctx) { def createUnresolvedTable( nsCtx: IdentifierReferenceContext, - patternCtx: StringLitContext): LogicalPlan = { + patternCtx: StringLitContext): LogicalPlan = withOrigin(patternCtx) { val ns = Option(nsCtx).map(x => visitMultipartIdentifier(x.multipartIdentifier())) UnresolvedTable(ns.getOrElse(Seq.empty[String]) :+ string(visitStringLit(patternCtx)), "SHOW TABLE EXTENDED ... PARTITION ...") 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 6bb175fc6de3..63077542d829 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 @@ -889,14 +889,13 @@ object ShowTables { case class ShowTablesExtended( namespace: LogicalPlan, pattern: String, - override val output: Seq[Attribute] = ShowTablesExtended.getOutputAttrs) - extends UnaryCommand { + override val output: Seq[Attribute] = ShowTablesUtils.getOutputAttrs) extends UnaryCommand { override def child: LogicalPlan = namespace override protected def withNewChildInternal(newChild: LogicalPlan): ShowTablesExtended = copy(namespace = newChild) } -object ShowTablesExtended { +object ShowTablesUtils { def getOutputAttrs: Seq[Attribute] = Seq( AttributeReference("namespace", StringType, nullable = false)(), AttributeReference("tableName", StringType, nullable = false)(), @@ -910,20 +909,12 @@ object ShowTablesExtended { case class ShowTablePartition( table: LogicalPlan, partitionSpec: PartitionSpec, - override val output: Seq[Attribute] = ShowTablePartition.getOutputAttrs) + override val output: Seq[Attribute] = ShowTablesUtils.getOutputAttrs) extends V2PartitionCommand { override protected def withNewChildInternal(newChild: LogicalPlan): ShowTablePartition = copy(table = newChild) } -object ShowTablePartition { - def getOutputAttrs: Seq[Attribute] = Seq( - AttributeReference("namespace", StringType, nullable = false)(), - AttributeReference("tableName", StringType, nullable = false)(), - AttributeReference("isTemporary", BooleanType, nullable = false)(), - AttributeReference("information", StringType, nullable = false)()) -} - /** * The logical plan of the SHOW VIEWS 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 bea7d9517937..b0472851fe16 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 @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.{SPARK_DOC_ROOT, SparkException, SparkThrowable, SparkThrowableHelper, SparkUnsupportedOperationException} import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, FunctionIdentifier, QualifiedTableName, TableIdentifier} +import org.apache.spark.sql.catalyst.{ExtendedAnalysisException, FunctionIdentifier, InternalRow, QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, FunctionAlreadyExistsException, NamespaceAlreadyExistsException, NoSuchFunctionException, NoSuchNamespaceException, NoSuchPartitionException, NoSuchTableException, ResolvedTable, Star, TableAlreadyExistsException, UnresolvedRegex} import org.apache.spark.sql.catalyst.catalog.{CatalogTable, InvalidUDFClassException} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec @@ -2535,6 +2535,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat new NoSuchPartitionException(db, table, partition) } + def notExistPartitionError( + table: Identifier, + partitionIdent: InternalRow, + partitionSchema: StructType): Throwable = { + new NoSuchPartitionException(table.toString, partitionIdent, partitionSchema) + } + def analyzingColumnStatisticsNotSupportedForColumnTypeError( name: String, dataType: DataType): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 02a383dccedd..afc244509c41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -33,8 +33,8 @@ import org.apache.spark._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.memory.SparkOutOfMemoryError import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, UnresolvedGenerator} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.parser.ParseException @@ -2785,11 +2785,4 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE "upper" -> size.toString, "invalidValue" -> pos.toString)) } - - def notExistPartitionError( - tableName: String, - partitionIdent: InternalRow, - partitionSchema: StructType): Throwable = { - new NoSuchPartitionException(tableName, partitionIdent, partitionSchema) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala index d29275f27dca..6b884713bd5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap object DataSourceV2Implicits { implicit class TableHelper(table: Table) { - def asReadable: SupportsRead = { table match { case support: SupportsRead => 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 e99327dc16eb..c222b149288d 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 @@ -245,7 +245,6 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) pattern, output) => val newOutput = if (conf.getConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA)) { - assert(output.length == 4) output.head.withName("database") +: output.tail } else { output @@ -257,7 +256,6 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) partitionSpec, output) if isSessionCatalog(catalog) => val newOutput = if (conf.getConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA)) { - assert(output.length == 4) output.head.withName("database") +: output.tail } else { output 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 7952d2d38da1..d719ff18ac6c 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 @@ -408,12 +408,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat ResolvedNamespace(catalog, ns), pattern, output) => - ShowTablesExec(output, catalog.asTableCatalog, ns, Some(pattern), - isExtended = true, partitionSpec = None) :: Nil + ShowTablesExtendedExec(output, catalog.asTableCatalog, ns, pattern) :: Nil case ShowTablePartition(r: ResolvedTable, part, output) => - ShowTablesExec(output, r.catalog, r.identifier.namespace(), Some(r.identifier.name()), - isExtended = true, partitionSpec = Some(Seq(part).asResolvedPartitionSpecs.head)) :: Nil + ShowTablePartitionExec(output, r.catalog, r.identifier, + r.table.asPartitionable, Seq(part).asResolvedPartitionSpecs.head) :: Nil case SetCatalogAndNamespace(ResolvedNamespace(catalog, ns)) => val catalogManager = session.sessionState.catalogManager diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 96076c301a91..cde3dfb7ead5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -17,22 +17,14 @@ package org.apache.spark.sql.execution.datasources.v2 -import scala.collection.mutable 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 -import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} -import org.apache.spark.sql.catalyst.util.{quoteIdentifier, StringUtils} -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, SupportsPartitionManagement, Table, TableCatalog} -import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ -import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.execution.LeafExecNode -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper -import org.apache.spark.sql.types.{StringType, StructType} /** * Physical plan node for showing tables. @@ -41,36 +33,17 @@ case class ShowTablesExec( output: Seq[Attribute], catalog: TableCatalog, namespace: Seq[String], - pattern: Option[String], - isExtended: Boolean = false, - partitionSpec: Option[ResolvedPartitionSpec] = None) extends V2CommandExec with LeafExecNode { + pattern: Option[String]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - if (partitionSpec.isEmpty) { - // Show the information of tables. - val identifiers = catalog.listTables(namespace.toArray) - identifiers.map { identifier => - if (pattern.forall(StringUtils.filterPattern(Seq(identifier.name()), _).nonEmpty)) { - val isTemp = isTempView(identifier) - if (isExtended) { - val table = catalog.loadTable(identifier) - val information = extendedTable(catalog.name, identifier, table) - rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), isTemp, - s"$information\n") - } else { - rows += toCatalystRow(identifier.namespace().quoted, identifier.name(), isTemp) - } - } + val tables = catalog.listTables(namespace.toArray) + tables.map { table => + if (pattern.map(StringUtils.filterPattern(Seq(table.name()), _).nonEmpty).getOrElse(true)) { + rows += toCatalystRow(table.namespace().quoted, table.name(), isTempView(table)) } - } else { - // Show the information of partitions. - val identifier = Identifier.of(namespace.toArray, pattern.get) - val table = catalog.loadTable(identifier) - val isTemp = isTempView(identifier) - val information = extendedPartition(identifier, table.asPartitionable, partitionSpec.get) - rows += toCatalystRow(namespace.quoted, identifier.name(), isTemp, s"$information\n") } + rows.toSeq } @@ -80,103 +53,4 @@ case class ShowTablesExec( case _ => false } } - - private def extendedTable( - catalogName: String, - identifier: Identifier, - table: Table): String = { - val results = new mutable.LinkedHashMap[String, String]() - - results.put("Catalog", catalogName) - - if (!identifier.namespace().isEmpty) { - results.put("Namespace", identifier.namespace().quoted) - } - results.put("Table", identifier.name()) - val tableType = if (table.properties().containsKey(TableCatalog.PROP_EXTERNAL)) { - CatalogTableType.EXTERNAL - } else { - CatalogTableType.MANAGED - } - results.put("Type", tableType.name) - - CatalogV2Util.TABLE_RESERVED_PROPERTIES - .filterNot(_ == TableCatalog.PROP_EXTERNAL) - .foreach(propKey => { - if (table.properties.containsKey(propKey)) { - results.put(propKey.capitalize, table.properties.get(propKey)) - } - }) - - val properties = - conf.redactOptions(table.properties.asScala.toMap).toList - .filter(kv => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(kv._1)) - .sortBy(_._1).map { - case (key, value) => key + "=" + value - }.mkString("[", ",", "]") - if (table.properties().isEmpty) { - results.put("Table Properties", properties.mkString("[", ", ", "]")) - } - - // Partition Provider & Partition Columns - var partitionColumns = new StructType() - if (table.supportsPartitions && table.asPartitionable.partitionSchema().nonEmpty) { - partitionColumns = table.asPartitionable.partitionSchema() - results.put("Partition Provider", "Catalog") - results.put("Partition Columns", table.asPartitionable.partitionSchema().map( - field => quoteIdentifier(field.name)).mkString("[", ", ", "]")) - } - - if (table.schema().nonEmpty) { - val dataColumns = table.schema().filterNot(partitionColumns.contains) - results.put("Schema", StructType(dataColumns ++ partitionColumns).treeString) - } - - results.map { case (key, value) => - if (value.isEmpty) key else s"$key: $value" - }.mkString("", "\n", "") - } - - private def extendedPartition( - identifier: Identifier, - partitionTable: SupportsPartitionManagement, - resolvedPartitionSpec: ResolvedPartitionSpec): String = { - val results = new mutable.LinkedHashMap[String, String]() - - // "Partition Values" - val partitionSchema = partitionTable.partitionSchema() - val (names, ident) = (resolvedPartitionSpec.names, resolvedPartitionSpec.ident) - val partitionIdentifiers = partitionTable.listPartitionIdentifiers(names.toArray, ident) - if (partitionIdentifiers.isEmpty) { - throw QueryExecutionErrors.notExistPartitionError(identifier.toString, ident, partitionSchema) - } - val row = partitionIdentifiers.head - 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 = - Cast(Literal(row.get(i, dataType), dataType), StringType, Some(timeZoneId)).eval() - val partValueStr = if (partValueUTF8String == null) "null" else partValueUTF8String.toString - partitions(i) = escapePathName(partitionSchema(i).name) + "=" + escapePathName(partValueStr) - } - val partitionValues = partitions.mkString("[", ", ", "]") - results.put("Partition Values", s"$partitionValues") - - // "Partition Parameters" - val metadata = partitionTable.loadPartitionMetadata(ident) - if (!metadata.isEmpty) { - val metadataValues = metadata.asScala.map { case (key, value) => - if (value.isEmpty) key else s"$key: $value" - }.mkString("{", ", ", "}") - results.put("Partition Parameters", metadataValues) - } - - // TODO "Created Time", "Last Access", "Partition Statistics" - - results.map { case (key, value) => - if (value.isEmpty) key else s"$key: $value" - }.mkString("", "\n", "") - } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out index 167e2f8622b9..ce5f7995f5d1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out @@ -130,13 +130,20 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1) -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchTableException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", "messageParameters" : { - "relationName" : "`showdb`.`show_t*`" - } + "relationName" : "`show_t*`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 34, + "fragment" : "'show_t*'" + } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index a37cf630969d..442f0fe5d5f2 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -208,13 +208,20 @@ SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1) -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", "sqlState" : "42P01", "messageParameters" : { - "relationName" : "`showdb`.`show_t*`" - } + "relationName" : "`show_t*`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 34, + "fragment" : "'show_t*'" + } ] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index 270cf4620760..a89d3f61ad4d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -184,11 +184,13 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(0) === namespace) - assert(result.collect()(0)(1) === table) - assert(result.collect()(0)(2) === false) - val actualResult = exclude(result.collect()(0)(3).toString) + + val resultCollect = result.collect() + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(0) === namespace) + assert(resultCollect(0)(1) === table) + assert(resultCollect(0)(2) === false) + val actualResult = exclude(resultCollect(0)(3).toString) val expectedResult = "Partition Values: [id1=1, id2=2]" assert(actualResult === expectedResult) } @@ -212,12 +214,14 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { .sort("tableName") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect().length == 3) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(1) === table) - assert(result.collect()(0)(2) === false) - val actualResult_0_3 = exclude(result.collect()(0)(3).toString) + val resultCollect = result.collect() + assert(resultCollect.length == 3) + + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(1) === table) + assert(resultCollect(0)(2) === false) + val actualResult_0_3 = exclude(resultCollect(0)(3).toString) // exclude "Created Time", "Last Access", "Created By", "Location" val expectedResult_0_3 = @@ -234,10 +238,10 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { assert(actualResult_0_3 === expectedResult_0_3) - assert(result.collect()(1).length == 4) - assert(result.collect()(1)(1) === table1) - assert(result.collect()(1)(2) === false) - val actualResult_1_3 = exclude(result.collect()(1)(3).toString) + assert(resultCollect(1).length == 4) + assert(resultCollect(1)(1) === table1) + assert(resultCollect(1)(2) === false) + val actualResult_1_3 = exclude(resultCollect(1)(3).toString) // exclude "Created Time", "Last Access", "Created By", "Location" val expectedResult_1_3 = @@ -253,10 +257,10 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { | |-- id1: long (nullable = true)""".stripMargin assert(actualResult_1_3 === expectedResult_1_3) - assert(result.collect()(2).length == 4) - assert(result.collect()(2)(1) === table2) - assert(result.collect()(2)(2) === false) - val actualResult_2_3 = exclude(result.collect()(2)(3).toString) + assert(resultCollect(2).length == 4) + assert(resultCollect(2)(1) === table2) + assert(resultCollect(2)(2) === false) + val actualResult_2_3 = exclude(resultCollect(2)(3).toString) // exclude "Created Time", "Last Access", "Created By", "Location" val expectedResult_2_3 = @@ -274,4 +278,98 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { } } } + + test("show table extended in view: permanent, temp global, temp local") { + val namespace = "ns" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + val viewName = table + "_view" + val tmpLocalViewName = viewName + "_local_tmp" + val tmpGlobalViewName = viewName + "_global_tmp" + withView(viewName, tmpLocalViewName) { + sql(s"CREATE VIEW $viewName AS SELECT id FROM $t") + sql(s"CREATE TEMPORARY VIEW $tmpLocalViewName AS SELECT id FROM $t") + + // permanent view + temp local view + val result1 = sql(s"SHOW TABLE EXTENDED LIKE '$viewName*'").sort("tableName") + + assert(result1.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + val resultCollect1 = result1.collect() + assert(resultCollect1.length == 2) + + assert(resultCollect1(0).length == 4) + assert(resultCollect1(0)(1) === viewName) + assert(resultCollect1(0)(2) === false) + val actualResult1_1 = exclude(resultCollect1(0)(3).toString) + val expectedResult1_1 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $viewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Original Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.$namespace + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult1_1 === expectedResult1_1) + + assert(resultCollect1(1).length == 4) + assert(resultCollect1(1)(1) === tmpLocalViewName) + assert(resultCollect1(1)(2) === true) + val actualResult1_2 = exclude(resultCollect1(1)(3).toString) + val expectedResult1_2 = + s"""Table: $tmpLocalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.$namespace + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult1_2 === expectedResult1_2) + + withGlobalView(tmpGlobalViewName) { + sql(s"CREATE GLOBAL TEMPORARY VIEW $tmpGlobalViewName AS SELECT id FROM $t") + + // permanent view + temp global view + val result2 = sql(s"SHOW TABLE EXTENDED in global_temp LIKE '$viewName*'"). + sort("tableName") + + val resultCollect2 = result2.collect() + assert(resultCollect2.length == 2) + + assert(resultCollect2(0).length == 4) + assert(resultCollect2(0)(1) === tmpGlobalViewName) + assert(resultCollect2(0)(2) === true) + val actualResult2_1 = exclude(resultCollect2(0)(3).toString) + val expectedResult2_1 = + s"""Database: global_temp + |Table: $tmpGlobalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.$namespace + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult2_1 === expectedResult2_1) + + assert(resultCollect2(1).length == 4) + assert(resultCollect2(1)(1) === tmpLocalViewName) + assert(resultCollect2(1)(2) === true) + val actualResult2_2 = exclude(resultCollect2(1)(3).toString) + val expectedResult2_2 = + s"""Table: $tmpLocalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.$namespace + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult2_2 === expectedResult2_2) + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index bd2478e6d324..f9dddf7a074d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -78,11 +78,12 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(0) === namespace) - assert(result.collect()(0)(1) === table) - assert(result.collect()(0)(2) === false) - val actualResult = exclude(result.collect()(0)(3).toString) + val resultCollect = result.collect() + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(0) === namespace) + assert(resultCollect(0)(1) === table) + assert(resultCollect(0)(2) === false) + val actualResult = exclude(resultCollect(0)(3).toString) val expectedResult = "Partition Values: [id1=1, id2=2]" assert(actualResult === expectedResult) } @@ -106,12 +107,13 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase .sort("tableName") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect().length == 3) + val resultCollect = result.collect() + assert(resultCollect.length == 3) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(1) === table) - assert(result.collect()(0)(2) === false) - val actualResult_0_3 = exclude(result.collect()(0)(3).toString) + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(1) === table) + assert(resultCollect(0)(2) === false) + val actualResult_0_3 = exclude(resultCollect(0)(3).toString) // exclude "Created Time", "Last Access", "Created By", "Location" val expectedResult_0_3 = @@ -128,10 +130,10 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase | |-- id: long (nullable = true)""".stripMargin assert(actualResult_0_3 === expectedResult_0_3) - assert(result.collect()(1).length == 4) - assert(result.collect()(1)(1) === table1) - assert(result.collect()(1)(2) === false) - val actualResult_1_3 = exclude(result.collect()(1)(3).toString) + assert(resultCollect(1).length == 4) + assert(resultCollect(1)(1) === table1) + assert(resultCollect(1)(2) === false) + val actualResult_1_3 = exclude(resultCollect(1)(3).toString) // exclude "Created Time", "Last Access", "Created By", "Location" val expectedResult_1_3 = @@ -148,10 +150,10 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase | |-- id1: long (nullable = true)""".stripMargin assert(actualResult_1_3 === expectedResult_1_3) - assert(result.collect()(2).length == 4) - assert(result.collect()(2)(1) === table2) - assert(result.collect()(2)(2) === false) - val actualResult_2_3 = exclude(result.collect()(2)(3).toString) + assert(resultCollect(2).length == 4) + assert(resultCollect(2)(1) === table2) + assert(resultCollect(2)(2) === false) + val actualResult_2_3 = exclude(resultCollect(2)(3).toString) // exclude "Created Time", "Last Access", "Created By", "Location" val expectedResult_2_3 = @@ -170,4 +172,97 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase } } } + + test("show table extended in view: permanent, temp global, temp local") { + val namespace = "ns" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + val viewName = table + "_view" + val tmpLocalViewName = viewName + "_local_tmp" + val tmpGlobalViewName = viewName + "_global_tmp" + withView(viewName, tmpLocalViewName) { + sql(s"CREATE VIEW $viewName AS SELECT id FROM $t") + sql(s"CREATE TEMPORARY VIEW $tmpLocalViewName AS SELECT id FROM $t") + + // permanent view + temp local view + val result1 = sql(s"SHOW TABLE EXTENDED LIKE '$viewName*'").sort("tableName") + + assert(result1.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + val resultCollect1 = result1.collect() + assert(resultCollect1.length == 2) + + assert(resultCollect1(0).length == 4) + assert(resultCollect1(0)(1) === viewName) + assert(resultCollect1(0)(2) === false) + val actualResult1_1 = exclude(resultCollect1(0)(3).toString) + val expectedResult1_1 = + s"""Catalog: spark_catalog + |Database: default + |Table: $viewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Original Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: spark_catalog.default + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult1_1 === expectedResult1_1) + + assert(resultCollect1(1).length == 4) + assert(resultCollect1(1)(1) === tmpLocalViewName) + assert(resultCollect1(1)(2) === true) + val actualResult1_2 = exclude(resultCollect1(1)(3).toString) + val expectedResult1_2 = + s"""Table: $tmpLocalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: spark_catalog.default + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult1_2 === expectedResult1_2) + + withGlobalView(tmpGlobalViewName) { + sql(s"CREATE GLOBAL TEMPORARY VIEW $tmpGlobalViewName AS SELECT id FROM $t") + + // permanent view + temp global view + val result2 = sql(s"SHOW TABLE EXTENDED in global_temp LIKE '$viewName*'"). + sort("tableName") + val resultCollect2 = result2.collect() + assert(resultCollect2.length == 2) + + assert(resultCollect2(0).length == 4) + assert(resultCollect2(0)(1) === tmpGlobalViewName) + assert(resultCollect2(0)(2) === true) + val actualResult2_1 = exclude(resultCollect2(0)(3).toString) + val expectedResult2_1 = + s"""Database: global_temp + |Table: $tmpGlobalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: spark_catalog.default + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult2_1 === expectedResult2_1) + + assert(resultCollect2(1).length == 4) + assert(resultCollect2(1)(1) === tmpLocalViewName) + assert(resultCollect2(1)(2) === true) + val actualResult2_2 = exclude(resultCollect2(1)(3).toString) + val expectedResult2_2 = + s"""Table: $tmpLocalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: spark_catalog.default + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult2_2 === expectedResult2_2) + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index fab5c49da8e4..4ae23cb307dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -320,6 +320,17 @@ private[sql] trait SQLTestUtilsBase ) } + /** + * Drops global view `viewName` after calling `f`. + */ + protected def withGlobalView(viewNames: String*)(f: => Unit): Unit = { + Utils.tryWithSafeFinally(f)( + viewNames.foreach { name => + spark.sessionState.catalog.dropGlobalTempView(name) + } + ) + } + /** * Drops cache `cacheName` after calling `f`. */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala index 7bf2e31915a7..73567ce64c1e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala @@ -65,11 +65,13 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(0) === namespace) - assert(result.collect()(0)(1) === table) - assert(result.collect()(0)(2) === false) - val actualResult = exclude(result.collect()(0)(3).toString) + + val resultCollect = result.collect() + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(0) === namespace) + assert(resultCollect(0)(1) === table) + assert(resultCollect(0)(2) === false) + val actualResult = exclude(resultCollect(0)(3).toString) val expectedResult = """Partition Values: [id1=1, id2=2] |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -98,12 +100,13 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { .sort("tableName") assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - assert(result.collect().length == 3) + val resultCollect = result.collect() + assert(resultCollect.length == 3) - assert(result.collect()(0).length == 4) - assert(result.collect()(0)(1) === table) - assert(result.collect()(0)(2) === false) - val actualResult_0_3 = exclude(result.collect()(0)(3).toString) + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(1) === table) + assert(resultCollect(0)(2) === false) + val actualResult_0_3 = exclude(resultCollect(0)(3).toString) // exclude "Table Properties" val expectedResult_0_3 = @@ -124,10 +127,10 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { | |-- id: long (nullable = true)""".stripMargin assert(actualResult_0_3 === expectedResult_0_3) - assert(result.collect()(1).length == 4) - assert(result.collect()(1)(1) === table1) - assert(result.collect()(1)(2) === false) - val actualResult_1_3 = exclude(result.collect()(1)(3).toString) + assert(resultCollect(1).length == 4) + assert(resultCollect(1)(1) === table1) + assert(resultCollect(1)(2) === false) + val actualResult_1_3 = exclude(resultCollect(1)(3).toString) // exclude "Table Properties" val expectedResult_1_3 = @@ -148,10 +151,10 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { | |-- id1: long (nullable = true)""".stripMargin assert(actualResult_1_3 === expectedResult_1_3) - assert(result.collect()(2).length == 4) - assert(result.collect()(2)(1) === table2) - assert(result.collect()(2)(2) === false) - val actualResult_2_3 = exclude(result.collect()(2)(3).toString) + assert(resultCollect(2).length == 4) + assert(resultCollect(2)(1) === table2) + assert(resultCollect(2)(2) === false) + val actualResult_2_3 = exclude(resultCollect(2)(3).toString) // exclude "Table Properties" val expectedResult_2_3 = @@ -174,4 +177,102 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { } } } + + test("show table extended in view: permanent, temp global, temp local") { + val namespace = "ns" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + val viewName = table + "_view" + val tmpLocalViewName = viewName + "_local_tmp" + val tmpGlobalViewName = viewName + "_global_tmp" + withView(viewName, tmpLocalViewName) { + sql(s"CREATE VIEW $viewName AS SELECT id FROM $t") + sql(s"CREATE TEMPORARY VIEW $tmpLocalViewName AS SELECT id FROM $t") + + // permanent view + temp local view + val result1 = sql(s"SHOW TABLE EXTENDED LIKE '$viewName*'").sort("tableName") + + assert(result1.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + val resultCollect1 = result1.collect() + assert(resultCollect1.length == 2) + + assert(resultCollect1(0).length == 4) + assert(resultCollect1(0)(1) === viewName) + assert(resultCollect1(0)(2) === false) + val actualResult1_1 = exclude(resultCollect1(0)(3).toString) + val expectedResult1_1 = + s"""Catalog: $catalog + |Database: $namespace + |Table: $viewName + |Owner: ${Utils.getCurrentUserName()} + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Original Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.$namespace + |View Query Output Columns: [id] + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + |Storage Properties: [serialization.format=1] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult1_1 === expectedResult1_1) + + assert(resultCollect1(1).length == 4) + assert(resultCollect1(1)(1) === tmpLocalViewName) + assert(resultCollect1(1)(2) === true) + val actualResult1_2 = exclude(resultCollect1(1)(3).toString) + val expectedResult1_2 = + s"""Table: $tmpLocalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.$namespace + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult1_2 === expectedResult1_2) + + withGlobalView(tmpGlobalViewName) { + sql(s"CREATE GLOBAL TEMPORARY VIEW $tmpGlobalViewName AS SELECT id FROM $t") + + // permanent view + temp global view + val result2 = sql(s"SHOW TABLE EXTENDED in global_temp LIKE '$viewName*'"). + sort("tableName") + val resultCollect2 = result2.collect() + assert(resultCollect2.length == 2) + + assert(resultCollect2(0).length == 4) + assert(resultCollect2(0)(1) === tmpGlobalViewName) + assert(resultCollect2(0)(2) === true) + val actualResult2_1 = exclude(resultCollect2(0)(3).toString) + val expectedResult2_1 = + s"""Database: global_temp + |Table: $tmpGlobalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.$namespace + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult2_1 === expectedResult2_1) + + assert(resultCollect2(1).length == 4) + assert(resultCollect2(1)(1) === tmpLocalViewName) + assert(resultCollect2(1)(2) === true) + val actualResult2_2 = exclude(resultCollect2(1)(3).toString) + val expectedResult2_2 = + s"""Table: $tmpLocalViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: $catalog.$namespace + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualResult2_2 === expectedResult2_2) + } + } + } + } } From bcb5db7ef098c39df17ff4b90017ccc86a45ea25 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 30 Oct 2023 13:26:15 +0800 Subject: [PATCH 17/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../v2/ShowTablesExtendedExec.scala | 201 ++++++++++++++++++ 1 file changed, 201 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala new file mode 100644 index 000000000000..b12740ad02da --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala @@ -0,0 +1,201 @@ +/* + * 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 +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 +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} +import org.apache.spark.sql.catalyst.util.{quoteIdentifier, StringUtils} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, SupportsPartitionManagement, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper +import org.apache.spark.sql.types.{StringType, StructType} + +/** + * Physical plan node for showing tables without partition, Show the information of tables. + */ +case class ShowTablesExtendedExec( + output: Seq[Attribute], + catalog: TableCatalog, + namespace: Seq[String], + pattern: String) extends V2CommandExec with LeafExecNode { + override protected def run(): Seq[InternalRow] = { + val rows = new ArrayBuffer[InternalRow]() + + // fetch tables + // TODO We need a new listTable overload that takes a pattern string. + val tables = catalog.listTables(namespace.toArray) + tables.map { tableIdent => + if (Option(pattern).forall(StringUtils.filterPattern( + Seq(tableIdent.name()), _).nonEmpty)) { + val table = catalog.loadTable(tableIdent) + val information = getTableDetails(catalog.name, tableIdent, table) + rows += toCatalystRow(tableIdent.namespace().quoted, tableIdent.name(), false, + s"$information\n") + } + } + + // fetch views, includes: view, global temp view, local temp view + val sessionCatalog = session.sessionState.catalog + val (namespaceExists, db) = namespace match { + case Seq(db) => + (sessionCatalog.databaseExists(db), Some(db)) + case _ => + (false, None) + } + if (namespaceExists) { + val views = sessionCatalog.listViews(db.get, pattern) + views.map { viewIdent => + val tableName = viewIdent.table + val isTemp = sessionCatalog.isTempView(viewIdent) + val view = sessionCatalog.getTempViewOrPermanentTableMetadata(viewIdent) + val information = view.simpleString + rows += toCatalystRow(db.get, tableName, isTemp, s"$information\n") + } + } + + rows.toSeq + } + + private def getTableDetails( + catalogName: String, + identifier: Identifier, + table: Table): String = { + val results = new mutable.LinkedHashMap[String, String]() + + results.put("Catalog", catalogName) + + if (!identifier.namespace().isEmpty) { + results.put("Namespace", identifier.namespace().quoted) + } + results.put("Table", identifier.name()) + val tableType = if (table.properties().containsKey(TableCatalog.PROP_EXTERNAL)) { + CatalogTableType.EXTERNAL + } else { + CatalogTableType.MANAGED + } + results.put("Type", tableType.name) + + CatalogV2Util.TABLE_RESERVED_PROPERTIES + .filterNot(_ == TableCatalog.PROP_EXTERNAL) + .foreach(propKey => { + if (table.properties.containsKey(propKey)) { + results.put(propKey.capitalize, table.properties.get(propKey)) + } + }) + + val properties = + conf.redactOptions(table.properties.asScala.toMap).toList + .filter(kv => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(kv._1)) + .sortBy(_._1).map { + case (key, value) => key + "=" + value + }.mkString("[", ",", "]") + if (table.properties().isEmpty) { + results.put("Table Properties", properties.mkString("[", ", ", "]")) + } + + // Partition Provider & Partition Columns + var partitionColumns = new StructType() + if (table.supportsPartitions && table.asPartitionable.partitionSchema().nonEmpty) { + partitionColumns = table.asPartitionable.partitionSchema() + results.put("Partition Provider", "Catalog") + results.put("Partition Columns", table.asPartitionable.partitionSchema().map( + field => quoteIdentifier(field.name)).mkString("[", ", ", "]")) + } + + if (table.schema().nonEmpty) { + val dataColumns = table.schema().filterNot(partitionColumns.contains) + results.put("Schema", StructType(dataColumns ++ partitionColumns).treeString) + } + + results.map { case (key, value) => + if (value.isEmpty) key else s"$key: $value" + }.mkString("", "\n", "") + } +} + +/** + * Physical plan node for showing tables with partition, Show the information of partitions. + */ +case class ShowTablePartitionExec( + output: Seq[Attribute], + catalog: TableCatalog, + tableIndent: Identifier, + table: SupportsPartitionManagement, + partSpec: ResolvedPartitionSpec) extends V2CommandExec with LeafExecNode { + override protected def run(): Seq[InternalRow] = { + val rows = new ArrayBuffer[InternalRow]() + val information = getTablePartitionDetails(tableIndent, + table, partSpec) + rows += toCatalystRow(tableIndent.namespace.quoted, + tableIndent.name(), false, s"$information\n") + + rows.toSeq + } + + private def getTablePartitionDetails( + tableIdent: Identifier, + partitionTable: SupportsPartitionManagement, + partSpec: ResolvedPartitionSpec): String = { + val results = new mutable.LinkedHashMap[String, String]() + + // "Partition Values" + val partitionSchema = partitionTable.partitionSchema() + val (names, ident) = (partSpec.names, partSpec.ident) + val partitionIdentifiers = partitionTable.listPartitionIdentifiers(names.toArray, ident) + if (partitionIdentifiers.isEmpty) { + throw QueryCompilationErrors.notExistPartitionError(tableIdent, ident, partitionSchema) + } + val row = partitionIdentifiers.head + 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 = + Cast(Literal(row.get(i, dataType), dataType), StringType, Some(timeZoneId)).eval() + val partValueStr = if (partValueUTF8String == null) "null" else partValueUTF8String.toString + partitions(i) = escapePathName(partitionSchema(i).name) + "=" + escapePathName(partValueStr) + } + val partitionValues = partitions.mkString("[", ", ", "]") + results.put("Partition Values", s"$partitionValues") + + // "Partition Parameters" + val metadata = partitionTable.loadPartitionMetadata(ident) + if (!metadata.isEmpty) { + val metadataValues = metadata.asScala.map { case (key, value) => + if (value.isEmpty) key else s"$key: $value" + }.mkString("{", ", ", "}") + results.put("Partition Parameters", metadataValues) + } + + // TODO "Created Time", "Last Access", "Partition Statistics" + + results.map { case (key, value) => + if (value.isEmpty) key else s"$key: $value" + }.mkString("", "\n", "") + } +} From 74e52ba9a421e6aaea401027e79ce2dfabde5a37 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 30 Oct 2023 19:34:33 +0800 Subject: [PATCH 18/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../sql/catalyst/catalog/SessionCatalog.scala | 16 ++++ .../v2/ShowTablesExtendedExec.scala | 35 +++---- .../command/ShowTablesSuiteBase.scala | 73 +++++++++++++++ .../command/v1/ShowTablesSuite.scala | 87 +++-------------- .../command/v2/ShowTablesSuite.scala | 93 ------------------- .../apache/spark/sql/test/SQLTestUtils.scala | 11 --- .../execution/command/ShowTablesSuite.scala | 86 +++-------------- 7 files changed, 129 insertions(+), 272 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e71865df94db..48570355d547 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1090,6 +1090,22 @@ class SessionCatalog( dbViews ++ listLocalTempViews(pattern) } + /** + * List all matching temp views in the specified database, including global/local temporary views. + */ + def listTempViews(db: String, pattern: String): Seq[TableIdentifier] = { + val dbName = format(db) + val dbViews = if (dbName == globalTempViewManager.database) { + globalTempViewManager.listViewNames(pattern).map { name => + TableIdentifier(name, Some(globalTempViewManager.database)) + } + } else { + Seq.empty + } + + dbViews ++ listLocalTempViews(pattern) + } + /** * List all matching local temporary views. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala index b12740ad02da..d7a45a32bfe8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala @@ -49,8 +49,7 @@ case class ShowTablesExtendedExec( // TODO We need a new listTable overload that takes a pattern string. val tables = catalog.listTables(namespace.toArray) tables.map { tableIdent => - if (Option(pattern).forall(StringUtils.filterPattern( - Seq(tableIdent.name()), _).nonEmpty)) { + if (StringUtils.filterPattern(Seq(tableIdent.name()), pattern).nonEmpty) { val table = catalog.loadTable(tableIdent) val information = getTableDetails(catalog.name, tableIdent, table) rows += toCatalystRow(tableIdent.namespace().quoted, tableIdent.name(), false, @@ -58,23 +57,20 @@ case class ShowTablesExtendedExec( } } - // fetch views, includes: view, global temp view, local temp view + // fetch temp views, includes: global temp view, local temp view val sessionCatalog = session.sessionState.catalog - val (namespaceExists, db) = namespace match { - case Seq(db) => - (sessionCatalog.databaseExists(db), Some(db)) - case _ => - (false, None) + val db = namespace match { + case Seq(db) => Some(db) + case _ => None } - if (namespaceExists) { - val views = sessionCatalog.listViews(db.get, pattern) - views.map { viewIdent => - val tableName = viewIdent.table - val isTemp = sessionCatalog.isTempView(viewIdent) - val view = sessionCatalog.getTempViewOrPermanentTableMetadata(viewIdent) - val information = view.simpleString - rows += toCatalystRow(db.get, tableName, isTemp, s"$information\n") - } + val views = sessionCatalog.listTempViews(db.get, pattern) + views.map { viewIdent => + val database = viewIdent.database.getOrElse("") + val tableName = viewIdent.table + val isTemp = sessionCatalog.isTempView(viewIdent) + val view = sessionCatalog.getTempViewOrPermanentTableMetadata(viewIdent) + val information = view.simpleString + rows += toCatalystRow(database, tableName, isTemp, s"$information\n") } rows.toSeq @@ -87,10 +83,7 @@ case class ShowTablesExtendedExec( val results = new mutable.LinkedHashMap[String, String]() results.put("Catalog", catalogName) - - if (!identifier.namespace().isEmpty) { - results.put("Namespace", identifier.namespace().quoted) - } + results.put("Namespace", identifier.namespace().quoted) results.put("Table", identifier.name()) val tableType = if (table.properties().containsKey(TableCatalog.PROP_EXTERNAL)) { CatalogTableType.EXTERNAL diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index c40475bd77e0..e8a92051698b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -198,6 +198,79 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } + test("show table extended in temp view, include: temp global, temp local") { + val namespace = "ns" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + val viewName = table + "_view" + val localTmpViewName = viewName + "_local_tmp" + val globalTmpViewName = viewName + "_global_tmp" + val globalNamespace = "global_temp" + withView(localTmpViewName, globalNamespace + "." + globalTmpViewName) { + sql(s"CREATE TEMPORARY VIEW $localTmpViewName AS SELECT id FROM $t") + sql(s"CREATE GLOBAL TEMPORARY VIEW $globalTmpViewName AS SELECT id FROM $t") + + // temp local view + val localResult = sql(s"SHOW TABLE EXTENDED LIKE '$viewName*'").sort("tableName") + assert(localResult.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + val localResultCollect = localResult.collect() + assert(localResultCollect.length == 1) + assert(localResultCollect(0).length == 4) + assert(localResultCollect(0)(1) === localTmpViewName) + assert(localResultCollect(0)(2) === true) + val actualLocalResult = exclude(localResultCollect(0)(3).toString) + val expectedLocalResult = + s"""Table: $localTmpViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: spark_catalog.default + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualLocalResult === expectedLocalResult) + + // temp global view + val globalResult = sql(s"SHOW TABLE EXTENDED in global_temp LIKE '$viewName*'"). + sort("tableName") + assert(globalResult.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + val globalResultCollect = globalResult.collect() + assert(globalResultCollect.length == 2) + + assert(globalResultCollect(0).length == 4) + assert(globalResultCollect(0)(1) === globalTmpViewName) + assert(globalResultCollect(0)(2) === true) + val actualGlobalResult1 = exclude(globalResultCollect(0)(3).toString) + val expectedGlobalResult1 = + s"""Database: $globalNamespace + |Table: $globalTmpViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: spark_catalog.default + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualGlobalResult1 === expectedGlobalResult1) + + assert(globalResultCollect(1).length == 4) + assert(globalResultCollect(1)(1) === localTmpViewName) + assert(globalResultCollect(1)(2) === true) + val actualLocalResult2 = exclude(globalResultCollect(1)(3).toString) + val expectedLocalResult2 = + s"""Table: $localTmpViewName + |Type: VIEW + |View Text: SELECT id FROM $catalog.$namespace.$table + |View Catalog and Namespace: spark_catalog.default + |View Query Output Columns: [id] + |Schema: root + | |-- id: integer (nullable = true)""".stripMargin + assert(actualLocalResult2 === expectedLocalResult2) + } + } + } + // Exclude some non-deterministic values for easy comparison of results, // such as `Created Time`, etc protected def exclude(text: String): String = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index a89d3f61ad4d..c38e22e98950 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -279,31 +279,24 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { } } - test("show table extended in view: permanent, temp global, temp local") { + test("show table extended in permanent view") { val namespace = "ns" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { t => sql(s"CREATE TABLE $t (id int) $defaultUsing") val viewName = table + "_view" - val tmpLocalViewName = viewName + "_local_tmp" - val tmpGlobalViewName = viewName + "_global_tmp" - withView(viewName, tmpLocalViewName) { - sql(s"CREATE VIEW $viewName AS SELECT id FROM $t") - sql(s"CREATE TEMPORARY VIEW $tmpLocalViewName AS SELECT id FROM $t") - - // permanent view + temp local view - val result1 = sql(s"SHOW TABLE EXTENDED LIKE '$viewName*'").sort("tableName") - - assert(result1.schema.fieldNames === + withView(viewName) { + sql(s"CREATE VIEW $catalog.$namespace.$viewName AS SELECT id FROM $t") + val result = sql(s"SHOW TABLE EXTENDED in $namespace LIKE '$viewName*'").sort("tableName") + assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - val resultCollect1 = result1.collect() - assert(resultCollect1.length == 2) - - assert(resultCollect1(0).length == 4) - assert(resultCollect1(0)(1) === viewName) - assert(resultCollect1(0)(2) === false) - val actualResult1_1 = exclude(resultCollect1(0)(3).toString) - val expectedResult1_1 = + val resultCollect = result.collect() + assert(resultCollect.length == 1) + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(1) === viewName) + assert(resultCollect(0)(2) === false) + val actualResult = exclude(resultCollect(0)(3).toString) + val expectedResult = s"""Catalog: $catalog |Database: $namespace |Table: $viewName @@ -314,61 +307,7 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { |View Query Output Columns: [id] |Schema: root | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult1_1 === expectedResult1_1) - - assert(resultCollect1(1).length == 4) - assert(resultCollect1(1)(1) === tmpLocalViewName) - assert(resultCollect1(1)(2) === true) - val actualResult1_2 = exclude(resultCollect1(1)(3).toString) - val expectedResult1_2 = - s"""Table: $tmpLocalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: $catalog.$namespace - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult1_2 === expectedResult1_2) - - withGlobalView(tmpGlobalViewName) { - sql(s"CREATE GLOBAL TEMPORARY VIEW $tmpGlobalViewName AS SELECT id FROM $t") - - // permanent view + temp global view - val result2 = sql(s"SHOW TABLE EXTENDED in global_temp LIKE '$viewName*'"). - sort("tableName") - - val resultCollect2 = result2.collect() - assert(resultCollect2.length == 2) - - assert(resultCollect2(0).length == 4) - assert(resultCollect2(0)(1) === tmpGlobalViewName) - assert(resultCollect2(0)(2) === true) - val actualResult2_1 = exclude(resultCollect2(0)(3).toString) - val expectedResult2_1 = - s"""Database: global_temp - |Table: $tmpGlobalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: $catalog.$namespace - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult2_1 === expectedResult2_1) - - assert(resultCollect2(1).length == 4) - assert(resultCollect2(1)(1) === tmpLocalViewName) - assert(resultCollect2(1)(2) === true) - val actualResult2_2 = exclude(resultCollect2(1)(3).toString) - val expectedResult2_2 = - s"""Table: $tmpLocalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: $catalog.$namespace - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult2_2 === expectedResult2_2) - } + assert(actualResult === expectedResult) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index f9dddf7a074d..19f6dca7a075 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -172,97 +172,4 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase } } } - - test("show table extended in view: permanent, temp global, temp local") { - val namespace = "ns" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { t => - sql(s"CREATE TABLE $t (id int) $defaultUsing") - val viewName = table + "_view" - val tmpLocalViewName = viewName + "_local_tmp" - val tmpGlobalViewName = viewName + "_global_tmp" - withView(viewName, tmpLocalViewName) { - sql(s"CREATE VIEW $viewName AS SELECT id FROM $t") - sql(s"CREATE TEMPORARY VIEW $tmpLocalViewName AS SELECT id FROM $t") - - // permanent view + temp local view - val result1 = sql(s"SHOW TABLE EXTENDED LIKE '$viewName*'").sort("tableName") - - assert(result1.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - val resultCollect1 = result1.collect() - assert(resultCollect1.length == 2) - - assert(resultCollect1(0).length == 4) - assert(resultCollect1(0)(1) === viewName) - assert(resultCollect1(0)(2) === false) - val actualResult1_1 = exclude(resultCollect1(0)(3).toString) - val expectedResult1_1 = - s"""Catalog: spark_catalog - |Database: default - |Table: $viewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Original Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: spark_catalog.default - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult1_1 === expectedResult1_1) - - assert(resultCollect1(1).length == 4) - assert(resultCollect1(1)(1) === tmpLocalViewName) - assert(resultCollect1(1)(2) === true) - val actualResult1_2 = exclude(resultCollect1(1)(3).toString) - val expectedResult1_2 = - s"""Table: $tmpLocalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: spark_catalog.default - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult1_2 === expectedResult1_2) - - withGlobalView(tmpGlobalViewName) { - sql(s"CREATE GLOBAL TEMPORARY VIEW $tmpGlobalViewName AS SELECT id FROM $t") - - // permanent view + temp global view - val result2 = sql(s"SHOW TABLE EXTENDED in global_temp LIKE '$viewName*'"). - sort("tableName") - val resultCollect2 = result2.collect() - assert(resultCollect2.length == 2) - - assert(resultCollect2(0).length == 4) - assert(resultCollect2(0)(1) === tmpGlobalViewName) - assert(resultCollect2(0)(2) === true) - val actualResult2_1 = exclude(resultCollect2(0)(3).toString) - val expectedResult2_1 = - s"""Database: global_temp - |Table: $tmpGlobalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: spark_catalog.default - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult2_1 === expectedResult2_1) - - assert(resultCollect2(1).length == 4) - assert(resultCollect2(1)(1) === tmpLocalViewName) - assert(resultCollect2(1)(2) === true) - val actualResult2_2 = exclude(resultCollect2(1)(3).toString) - val expectedResult2_2 = - s"""Table: $tmpLocalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: spark_catalog.default - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult2_2 === expectedResult2_2) - } - } - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index 4ae23cb307dd..fab5c49da8e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -320,17 +320,6 @@ private[sql] trait SQLTestUtilsBase ) } - /** - * Drops global view `viewName` after calling `f`. - */ - protected def withGlobalView(viewNames: String*)(f: => Unit): Unit = { - Utils.tryWithSafeFinally(f)( - viewNames.foreach { name => - spark.sessionState.catalog.dropGlobalTempView(name) - } - ) - } - /** * Drops cache `cacheName` after calling `f`. */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala index 73567ce64c1e..14e941aee082 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala @@ -178,31 +178,24 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { } } - test("show table extended in view: permanent, temp global, temp local") { + test("show table extended in permanent view") { val namespace = "ns" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { t => sql(s"CREATE TABLE $t (id int) $defaultUsing") val viewName = table + "_view" - val tmpLocalViewName = viewName + "_local_tmp" - val tmpGlobalViewName = viewName + "_global_tmp" - withView(viewName, tmpLocalViewName) { - sql(s"CREATE VIEW $viewName AS SELECT id FROM $t") - sql(s"CREATE TEMPORARY VIEW $tmpLocalViewName AS SELECT id FROM $t") - - // permanent view + temp local view - val result1 = sql(s"SHOW TABLE EXTENDED LIKE '$viewName*'").sort("tableName") - - assert(result1.schema.fieldNames === + withView(viewName) { + sql(s"CREATE VIEW $catalog.$namespace.$viewName AS SELECT id FROM $t") + val result = sql(s"SHOW TABLE EXTENDED in $namespace LIKE '$viewName*'").sort("tableName") + assert(result.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) - val resultCollect1 = result1.collect() - assert(resultCollect1.length == 2) - - assert(resultCollect1(0).length == 4) - assert(resultCollect1(0)(1) === viewName) - assert(resultCollect1(0)(2) === false) - val actualResult1_1 = exclude(resultCollect1(0)(3).toString) - val expectedResult1_1 = + val resultCollect = result.collect() + assert(resultCollect.length == 1) + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(1) === viewName) + assert(resultCollect(0)(2) === false) + val actualResult = exclude(resultCollect(0)(3).toString) + val expectedResult = s"""Catalog: $catalog |Database: $namespace |Table: $viewName @@ -218,60 +211,7 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { |Storage Properties: [serialization.format=1] |Schema: root | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult1_1 === expectedResult1_1) - - assert(resultCollect1(1).length == 4) - assert(resultCollect1(1)(1) === tmpLocalViewName) - assert(resultCollect1(1)(2) === true) - val actualResult1_2 = exclude(resultCollect1(1)(3).toString) - val expectedResult1_2 = - s"""Table: $tmpLocalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: $catalog.$namespace - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult1_2 === expectedResult1_2) - - withGlobalView(tmpGlobalViewName) { - sql(s"CREATE GLOBAL TEMPORARY VIEW $tmpGlobalViewName AS SELECT id FROM $t") - - // permanent view + temp global view - val result2 = sql(s"SHOW TABLE EXTENDED in global_temp LIKE '$viewName*'"). - sort("tableName") - val resultCollect2 = result2.collect() - assert(resultCollect2.length == 2) - - assert(resultCollect2(0).length == 4) - assert(resultCollect2(0)(1) === tmpGlobalViewName) - assert(resultCollect2(0)(2) === true) - val actualResult2_1 = exclude(resultCollect2(0)(3).toString) - val expectedResult2_1 = - s"""Database: global_temp - |Table: $tmpGlobalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: $catalog.$namespace - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult2_1 === expectedResult2_1) - - assert(resultCollect2(1).length == 4) - assert(resultCollect2(1)(1) === tmpLocalViewName) - assert(resultCollect2(1)(2) === true) - val actualResult2_2 = exclude(resultCollect2(1)(3).toString) - val expectedResult2_2 = - s"""Table: $tmpLocalViewName - |Type: VIEW - |View Text: SELECT id FROM $catalog.$namespace.$table - |View Catalog and Namespace: $catalog.$namespace - |View Query Output Columns: [id] - |Schema: root - | |-- id: integer (nullable = true)""".stripMargin - assert(actualResult2_2 === expectedResult2_2) - } + assert(actualResult === expectedResult) } } } From 723e66449b3b5266907e8cb91d15b1ac42ec57b9 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 1 Nov 2023 19:37:13 +0800 Subject: [PATCH 19/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../command/ShowTablesSuiteBase.scala | 156 +++++++++++++-- .../command/v1/ShowTablesSuite.scala | 146 +++------------ .../command/v2/ShowTablesSuite.scala | 133 ++----------- .../execution/command/ShowTablesSuite.scala | 177 ++++-------------- 4 files changed, 215 insertions(+), 397 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index e8a92051698b..f70a0775568f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -40,6 +40,38 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { checkAnswer(df, expected) } + // the error class & error parameters of + // `SHOW TABLE EXTENDED ... PARTITION ... in non-partitioned table` + protected def extendedPartInNonPartedTableError( + catalog: String, + namespace: String, + table: String): (String, Map[String, String]) + + protected def extendedPartExpectedResult: String = + "Partition Values: [id1=1, id2=2]" + + protected def namespaceKey: String = "Database" + + protected def extendedTableExpectedResultDiff: String + + private def extendedTableExpectedResult( + catalog: String, + namespaceName: String, + namespace: String, + table: String, + partColName: String, + dataColName: String): String = { + s"""Catalog: $catalog + |$namespaceName: $namespace + |Table: $table + |$extendedTableExpectedResultDiff + |Partition Provider: Catalog + |Partition Columns: [`$partColName`] + |Schema: root + | |-- $dataColName: string (nullable = true) + | |-- $partColName: long (nullable = true)""".stripMargin + } + test("show an existing table") { withNamespaceAndTable("ns", "table") { t => sql(s"CREATE TABLE $t (name STRING, id INT) $defaultUsing") @@ -175,6 +207,42 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } + test("show table extended in non-partitioned table") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") + val e = intercept[AnalysisException] { + sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") + } + val (errorClass, parameters) = extendedPartInNonPartedTableError(catalog, namespace, table) + checkError(exception = e, errorClass = errorClass, parameters = parameters) + } + } + + test("show table extended in multi partition key - " + + "the command's partition parameters are complete") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { tbl => + sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id1, id2)") + sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + + s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + val resultCollect = result.collect() + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(0) === namespace) + assert(resultCollect(0)(1) === table) + assert(resultCollect(0)(2) === false) + val actualResult = replace(resultCollect(0)(3).toString) + assert(actualResult === extendedPartExpectedResult) + } + } + test("show table extended in multi partition key - " + "the command's partition parameters are incomplete") { val namespace = "ns1" @@ -198,6 +266,57 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } + test("show table extended in multi tables") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { _ => + sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id)") + val table1 = "tbl1" + val table2 = "tbl2" + withTable(table1, table2) { + sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + + s"$defaultUsing PARTITIONED BY (id1)") + sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + + s"$defaultUsing PARTITIONED BY (id2)") + + val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") + .sort("tableName") + assert(result.schema.fieldNames === + Seq("namespace", "tableName", "isTemporary", "information")) + val resultCollect = result.collect() + assert(resultCollect.length == 3) + + assert(resultCollect(0).length == 4) + assert(resultCollect(0)(1) === table) + assert(resultCollect(0)(2) === false) + // replace "Created Time", "Last Access", "Created By", "Location" + val actualResult_0_3 = replace(resultCollect(0)(3).toString) + val expectedResult_0_3 = extendedTableExpectedResult( + catalog, namespaceKey, namespace, table, "id", "data") + assert(actualResult_0_3 === expectedResult_0_3) + + assert(resultCollect(1).length == 4) + assert(resultCollect(1)(1) === table1) + assert(resultCollect(1)(2) === false) + val actualResult_1_3 = replace(resultCollect(1)(3).toString) + // replace "Table Properties" + val expectedResult_1_3 = extendedTableExpectedResult( + catalog, namespaceKey, namespace, table1, "id1", "data1") + assert(actualResult_1_3 === expectedResult_1_3) + + assert(resultCollect(2).length == 4) + assert(resultCollect(2)(1) === table2) + assert(resultCollect(2)(2) === false) + val actualResult_2_3 = replace(resultCollect(2)(3).toString) + // replace "Table Properties" + val expectedResult_2_3 = extendedTableExpectedResult( + catalog, namespaceKey, namespace, table2, "id2", "data2") + assert(actualResult_2_3 === expectedResult_2_3) + } + } + } + test("show table extended in temp view, include: temp global, temp local") { val namespace = "ns" val table = "tbl" @@ -220,9 +339,12 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { assert(localResultCollect(0).length == 4) assert(localResultCollect(0)(1) === localTmpViewName) assert(localResultCollect(0)(2) === true) - val actualLocalResult = exclude(localResultCollect(0)(3).toString) + val actualLocalResult = replace(localResultCollect(0)(3).toString) val expectedLocalResult = s"""Table: $localTmpViewName + |Created Time: + |Last Access: + |Created By: |Type: VIEW |View Text: SELECT id FROM $catalog.$namespace.$table |View Catalog and Namespace: spark_catalog.default @@ -242,10 +364,13 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { assert(globalResultCollect(0).length == 4) assert(globalResultCollect(0)(1) === globalTmpViewName) assert(globalResultCollect(0)(2) === true) - val actualGlobalResult1 = exclude(globalResultCollect(0)(3).toString) + val actualGlobalResult1 = replace(globalResultCollect(0)(3).toString) val expectedGlobalResult1 = s"""Database: $globalNamespace |Table: $globalTmpViewName + |Created Time: + |Last Access: + |Created By: |Type: VIEW |View Text: SELECT id FROM $catalog.$namespace.$table |View Catalog and Namespace: spark_catalog.default @@ -257,9 +382,12 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { assert(globalResultCollect(1).length == 4) assert(globalResultCollect(1)(1) === localTmpViewName) assert(globalResultCollect(1)(2) === true) - val actualLocalResult2 = exclude(globalResultCollect(1)(3).toString) + val actualLocalResult2 = replace(globalResultCollect(1)(3).toString) val expectedLocalResult2 = s"""Table: $localTmpViewName + |Created Time: + |Last Access: + |Created By: |Type: VIEW |View Text: SELECT id FROM $catalog.$namespace.$table |View Catalog and Namespace: spark_catalog.default @@ -271,15 +399,17 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } - // Exclude some non-deterministic values for easy comparison of results, - // such as `Created Time`, etc - protected def exclude(text: String): String = { - text.split("\n").filter(line => - !line.startsWith("Created Time:") && - !line.startsWith("Last Access:") && - !line.startsWith("Created By:") && - !line.startsWith("Location:") && - !line.startsWith("Table Properties:") && - !line.startsWith("Partition Parameters:")).mkString("\n") + // Replace some non-deterministic values with deterministic value + // for easy comparison of results, such as `Created Time`, etc + protected def replace(text: String): String = { + text.split("\n").map { + case s"Created Time:$_" => "Created Time: " + case s"Last Access:$_" => "Last Access: " + case s"Created By:$_" => "Created By: " + case s"Location:$_" => "Location: " + case s"Table Properties:$_" => "Table Properties: " + case s"Partition Parameters:$_" => "Partition Parameters: " + case other => other + }.mkString("\n") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index c38e22e98950..a62f82515dc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -155,129 +155,28 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { } } - test("show table extended in non-partitioned table") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") - val e = intercept[AnalysisException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") - } - checkError( - exception = e, - errorClass = "_LEGACY_ERROR_TEMP_1251", - parameters = Map("action" -> "SHOW TABLE EXTENDED", "tableName" -> table) - ) - } + override protected def extendedPartInNonPartedTableError( + catalog: String, + namespace: String, + table: String): (String, Map[String, String]) = { + ("_LEGACY_ERROR_TEMP_1251", + Map("action" -> "SHOW TABLE EXTENDED", "tableName" -> table)) } - test("show table extended in multi partition key - " + - "the command's partition parameters are complete") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id1, id2)") - sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - - val resultCollect = result.collect() - assert(resultCollect(0).length == 4) - assert(resultCollect(0)(0) === namespace) - assert(resultCollect(0)(1) === table) - assert(resultCollect(0)(2) === false) - val actualResult = exclude(resultCollect(0)(3).toString) - val expectedResult = "Partition Values: [id1=1, id2=2]" - assert(actualResult === expectedResult) - } - } - - test("show table extended in multi tables") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { _ => - sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id)") - val table1 = "tbl1" - val table2 = "tbl2" - withTable(table1, table2) { - sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + - s"$defaultUsing PARTITIONED BY (id1)") - sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + - s"$defaultUsing PARTITIONED BY (id2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") - .sort("tableName") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - - val resultCollect = result.collect() - assert(resultCollect.length == 3) - - assert(resultCollect(0).length == 4) - assert(resultCollect(0)(1) === table) - assert(resultCollect(0)(2) === false) - val actualResult_0_3 = exclude(resultCollect(0)(3).toString) - - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_0_3 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table - |Type: MANAGED - |Provider: parquet - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - - assert(actualResult_0_3 === expectedResult_0_3) - - assert(resultCollect(1).length == 4) - assert(resultCollect(1)(1) === table1) - assert(resultCollect(1)(2) === false) - val actualResult_1_3 = exclude(resultCollect(1)(3).toString) - - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_1_3 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table1 - |Type: MANAGED - |Provider: parquet - |Partition Provider: Catalog - |Partition Columns: [`id1`] - |Schema: root - | |-- data1: string (nullable = true) - | |-- id1: long (nullable = true)""".stripMargin - assert(actualResult_1_3 === expectedResult_1_3) - - assert(resultCollect(2).length == 4) - assert(resultCollect(2)(1) === table2) - assert(resultCollect(2)(2) === false) - val actualResult_2_3 = exclude(resultCollect(2)(3).toString) - - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_2_3 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table2 - |Type: MANAGED - |Provider: parquet - |Partition Provider: Catalog - |Partition Columns: [`id2`] - |Schema: root - | |-- data2: string (nullable = true) - | |-- id2: long (nullable = true)""".stripMargin - assert(actualResult_2_3 === expectedResult_2_3) - } - } - } + protected override def extendedPartExpectedResult: String = + super.extendedPartExpectedResult + + """ + |Location: + |Created Time: + |Last Access: """.stripMargin + + protected override def extendedTableExpectedResultDiff: String = + """Created Time: + |Last Access: + |Created By: + |Type: MANAGED + |Provider: parquet + |Location: """.stripMargin test("show table extended in permanent view") { val namespace = "ns" @@ -295,11 +194,14 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { assert(resultCollect(0).length == 4) assert(resultCollect(0)(1) === viewName) assert(resultCollect(0)(2) === false) - val actualResult = exclude(resultCollect(0)(3).toString) + val actualResult = replace(resultCollect(0)(3).toString) val expectedResult = s"""Catalog: $catalog |Database: $namespace |Table: $viewName + |Created Time: + |Last Access: + |Created By: |Type: VIEW |View Text: SELECT id FROM $catalog.$namespace.$table |View Original Text: SELECT id FROM $catalog.$namespace.$table diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index 19f6dca7a075..90ad16c4a4d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command.v2 -import org.apache.spark.sql.{AnalysisException, Row} +import org.apache.spark.sql.Row import org.apache.spark.sql.execution.command import org.apache.spark.util.Utils @@ -49,127 +49,18 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase } } - test("show table extended in non-partitioned table") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") - val e = intercept[AnalysisException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") - } - checkError( - exception = e, - errorClass = "_LEGACY_ERROR_TEMP_1231", - parameters = Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`") - ) - } + override protected def extendedPartInNonPartedTableError( + catalog: String, + namespace: String, + table: String): (String, Map[String, String]) = { + ("_LEGACY_ERROR_TEMP_1231", + Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`")) } - test("show table extended in multi partition key - " + - "the command's partition parameters are complete") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id1, id2)") - sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - val resultCollect = result.collect() - assert(resultCollect(0).length == 4) - assert(resultCollect(0)(0) === namespace) - assert(resultCollect(0)(1) === table) - assert(resultCollect(0)(2) === false) - val actualResult = exclude(resultCollect(0)(3).toString) - val expectedResult = "Partition Values: [id1=1, id2=2]" - assert(actualResult === expectedResult) - } - } - - test("show table extended in multi tables") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { _ => - sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id)") - val table1 = "tbl1" - val table2 = "tbl2" - withTable(table1, table2) { - sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + - s"$defaultUsing PARTITIONED BY (id1)") - sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + - s"$defaultUsing PARTITIONED BY (id2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") - .sort("tableName") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - val resultCollect = result.collect() - assert(resultCollect.length == 3) - - assert(resultCollect(0).length == 4) - assert(resultCollect(0)(1) === table) - assert(resultCollect(0)(2) === false) - val actualResult_0_3 = exclude(resultCollect(0)(3).toString) + protected override def namespaceKey: String = "Namespace" - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_0_3 = - s"""Catalog: $catalog - |Namespace: $namespace - |Table: $table - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - assert(actualResult_0_3 === expectedResult_0_3) - - assert(resultCollect(1).length == 4) - assert(resultCollect(1)(1) === table1) - assert(resultCollect(1)(2) === false) - val actualResult_1_3 = exclude(resultCollect(1)(3).toString) - - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_1_3 = - s"""Catalog: $catalog - |Namespace: $namespace - |Table: $table1 - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: [`id1`] - |Schema: root - | |-- data1: string (nullable = true) - | |-- id1: long (nullable = true)""".stripMargin - assert(actualResult_1_3 === expectedResult_1_3) - - assert(resultCollect(2).length == 4) - assert(resultCollect(2)(1) === table2) - assert(resultCollect(2)(2) === false) - val actualResult_2_3 = exclude(resultCollect(2)(3).toString) - - // exclude "Created Time", "Last Access", "Created By", "Location" - val expectedResult_2_3 = - s"""Catalog: $catalog - |Namespace: $namespace - |Table: $table2 - |Type: MANAGED - |Provider: _ - |Owner: ${Utils.getCurrentUserName()} - |Partition Provider: Catalog - |Partition Columns: [`id2`] - |Schema: root - | |-- data2: string (nullable = true) - | |-- id2: long (nullable = true)""".stripMargin - assert(actualResult_2_3 === expectedResult_2_3) - } - } - } + protected override def extendedTableExpectedResultDiff: String = + s"""Type: MANAGED + |Provider: _ + |Owner: ${Utils.getCurrentUserName()}""".stripMargin } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala index 14e941aee082..1a7d9c4b4175 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive.execution.command -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.command.v1 import org.apache.spark.util.Utils @@ -36,147 +35,39 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { } } - test("show table extended in non-partitioned table") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") - val e = intercept[AnalysisException] { - sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") - } - checkError( - exception = e, - errorClass = "_LEGACY_ERROR_TEMP_1231", - parameters = Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`") - ) - } + override protected def extendedPartInNonPartedTableError( + catalog: String, + namespace: String, + table: String): (String, Map[String, String]) = { + ("_LEGACY_ERROR_TEMP_1231", + Map("key" -> "id", "tblName" -> s"`$catalog`.`$namespace`.`$table`")) } - test("show table extended in multi partition key - " + - "the command's partition parameters are complete") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id1, id2)") - sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace " + - s"LIKE '$table' PARTITION(id1 = 1, id2 = 2)") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - - val resultCollect = result.collect() - assert(resultCollect(0).length == 4) - assert(resultCollect(0)(0) === namespace) - assert(resultCollect(0)(1) === table) - assert(resultCollect(0)(2) === false) - val actualResult = exclude(resultCollect(0)(3).toString) - val expectedResult = - """Partition Values: [id1=1, id2=2] - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1]""".stripMargin - assert(actualResult === expectedResult) - } - } - - test("show table extended in multi tables") { - val namespace = "ns1" - val table = "tbl" - withNamespaceAndTable(namespace, table, catalog) { _ => - sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + - s"$defaultUsing PARTITIONED BY (id)") - val table1 = "tbl1" - val table2 = "tbl2" - withTable(table1, table2) { - sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + - s"$defaultUsing PARTITIONED BY (id1)") - sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + - s"$defaultUsing PARTITIONED BY (id2)") - - val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") - .sort("tableName") - assert(result.schema.fieldNames === - Seq("namespace", "tableName", "isTemporary", "information")) - val resultCollect = result.collect() - assert(resultCollect.length == 3) - - assert(resultCollect(0).length == 4) - assert(resultCollect(0)(1) === table) - assert(resultCollect(0)(2) === false) - val actualResult_0_3 = exclude(resultCollect(0)(3).toString) - - // exclude "Table Properties" - val expectedResult_0_3 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table - |Owner: ${Utils.getCurrentUserName()} - |Type: MANAGED - |Provider: hive - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1] - |Partition Provider: Catalog - |Partition Columns: [`id`] - |Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true)""".stripMargin - assert(actualResult_0_3 === expectedResult_0_3) - - assert(resultCollect(1).length == 4) - assert(resultCollect(1)(1) === table1) - assert(resultCollect(1)(2) === false) - val actualResult_1_3 = exclude(resultCollect(1)(3).toString) - - // exclude "Table Properties" - val expectedResult_1_3 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table1 - |Owner: ${Utils.getCurrentUserName()} - |Type: MANAGED - |Provider: hive - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1] - |Partition Provider: Catalog - |Partition Columns: [`id1`] - |Schema: root - | |-- data1: string (nullable = true) - | |-- id1: long (nullable = true)""".stripMargin - assert(actualResult_1_3 === expectedResult_1_3) - - assert(resultCollect(2).length == 4) - assert(resultCollect(2)(1) === table2) - assert(resultCollect(2)(2) === false) - val actualResult_2_3 = exclude(resultCollect(2)(3).toString) - - // exclude "Table Properties" - val expectedResult_2_3 = - s"""Catalog: $catalog - |Database: $namespace - |Table: $table2 - |Owner: ${Utils.getCurrentUserName()} - |Type: MANAGED - |Provider: hive - |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - |InputFormat: org.apache.hadoop.mapred.TextInputFormat - |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - |Storage Properties: [serialization.format=1] - |Partition Provider: Catalog - |Partition Columns: [`id2`] - |Schema: root - | |-- data2: string (nullable = true) - | |-- id2: long (nullable = true)""".stripMargin - assert(actualResult_2_3 === expectedResult_2_3) - } - } - } + protected override def extendedPartExpectedResult: String = + super.extendedPartExpectedResult + + """ + |Location: + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1] + |Partition Parameters: + |Created Time: + |Last Access: """.stripMargin + + protected override def extendedTableExpectedResultDiff: String = + s"""Owner: ${Utils.getCurrentUserName()} + |Created Time: + |Last Access: + |Created By: + |Type: MANAGED + |Provider: hive + |Table Properties:
+ |Location: + |Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + |InputFormat: org.apache.hadoop.mapred.TextInputFormat + |OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + |Storage Properties: [serialization.format=1]""".stripMargin test("show table extended in permanent view") { val namespace = "ns" @@ -194,17 +85,21 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { assert(resultCollect(0).length == 4) assert(resultCollect(0)(1) === viewName) assert(resultCollect(0)(2) === false) - val actualResult = exclude(resultCollect(0)(3).toString) + val actualResult = replace(resultCollect(0)(3).toString) val expectedResult = s"""Catalog: $catalog |Database: $namespace |Table: $viewName |Owner: ${Utils.getCurrentUserName()} + |Created Time: + |Last Access: + |Created By: |Type: VIEW |View Text: SELECT id FROM $catalog.$namespace.$table |View Original Text: SELECT id FROM $catalog.$namespace.$table |View Catalog and Namespace: $catalog.$namespace |View Query Output Columns: [id] + |Table Properties:
|Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe |InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat |OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat From e06f867540b3b4d3bce769ec9c08f5712749c4dd Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 1 Nov 2023 19:58:17 +0800 Subject: [PATCH 20/29] Trigger build From 4af1c5e8c639344ada961c536435db987454e299 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 1 Nov 2023 22:02:26 +0800 Subject: [PATCH 21/29] Trigger build From ee254f4db4247c0af484f5a46ea34c34961048a6 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 2 Nov 2023 06:23:01 +0800 Subject: [PATCH 22/29] Trigger build From 02065eb3e6c9ee28e2380d2b3126d09829779b46 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 2 Nov 2023 09:06:11 +0800 Subject: [PATCH 23/29] debug appveyor --- dev/appveyor-install-dependencies.ps1 | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index f410e8774d96..785cc0ee5502 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -32,6 +32,7 @@ Function InstallR { } $rurl = $CRAN + "/bin/windows/base/" + $urlPath + "R-" + $rVer + "-win.exe" + echo $rurl # Downloading R Start-FileDownload $rurl "R-win.exe" From 9c3940cab4922e44eb0a0a8cca6fe0159148b9a3 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 2 Nov 2023 09:15:01 +0800 Subject: [PATCH 24/29] Temporarily set R-win version to 4.3.2 --- dev/appveyor-install-dependencies.ps1 | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index 785cc0ee5502..d6382aaec019 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -27,6 +27,7 @@ Function InstallR { $urlPath = "" $latestVer = $(ConvertFrom-JSON $(Invoke-WebRequest https://rversions.r-pkg.org/r-release-win).Content).version + $latestVer = "4.3.2" If ($rVer -ne $latestVer) { $urlPath = ("old/" + $rVer + "/") } From b47e6198f06fd033f2220abe3fd7aa8de0df731e Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 3 Nov 2023 09:47:10 +0800 Subject: [PATCH 25/29] Apply suggestions from code review Co-authored-by: Wenchen Fan --- .../spark/sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../execution/datasources/v2/ShowTablesExtendedExec.scala | 2 +- .../spark/sql/execution/command/ShowTablesSuiteBase.scala | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 48570355d547..a1c2797cef6f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1095,7 +1095,7 @@ class SessionCatalog( */ def listTempViews(db: String, pattern: String): Seq[TableIdentifier] = { val dbName = format(db) - val dbViews = if (dbName == globalTempViewManager.database) { + val globalTempViews = if (dbName == globalTempViewManager.database) { globalTempViewManager.listViewNames(pattern).map { name => TableIdentifier(name, Some(globalTempViewManager.database)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala index d7a45a32bfe8..8c70e7a55e12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala @@ -63,7 +63,7 @@ case class ShowTablesExtendedExec( case Seq(db) => Some(db) case _ => None } - val views = sessionCatalog.listTempViews(db.get, pattern) + val views = sessionCatalog.listTempViews(db.getOrElse(""), pattern) views.map { viewIdent => val database = viewIdent.database.getOrElse("") val tableName = viewIdent.table diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index f70a0775568f..5672a06d2762 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -177,7 +177,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { parameters = Map("schemaName" -> "`nonexist`")) } - test("show table extended in a not existing table") { + test("show table extended with no matching table") { val namespace = "ns1" val table = "nonexist" withNamespaceAndTable(namespace, table, catalog) { _ => @@ -188,7 +188,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } - test("show table extended in a not existing partition") { + test("show table extended with a not existing partition") { val namespace = "ns1" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { tbl => @@ -317,7 +317,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { } } - test("show table extended in temp view, include: temp global, temp local") { + test("show table extended with temp views") { val namespace = "ns" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { t => @@ -354,7 +354,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { assert(actualLocalResult === expectedLocalResult) // temp global view - val globalResult = sql(s"SHOW TABLE EXTENDED in global_temp LIKE '$viewName*'"). + val globalResult = sql(s"SHOW TABLE EXTENDED IN global_temp LIKE '$viewName*'"). sort("tableName") assert(globalResult.schema.fieldNames === Seq("namespace", "tableName", "isTemporary", "information")) From be5b337de22bd277a2ccaf8a0b7a0bcebd500be4 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 3 Nov 2023 20:41:33 +0800 Subject: [PATCH 26/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- .../sql/catalyst/catalog/SessionCatalog.scala | 14 ++-- .../sql/catalyst/parser/AstBuilder.scala | 16 +++-- .../v2/ShowTablesExtendedExec.scala | 25 ++++---- .../command/ShowTablesSuiteBase.scala | 64 +++++++++++++++++-- .../command/v1/ShowTablesSuite.scala | 2 +- .../command/v2/ShowTablesSuite.scala | 7 +- .../execution/command/ShowTablesSuite.scala | 2 +- 7 files changed, 97 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index a1c2797cef6f..e53b29716325 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1093,17 +1093,23 @@ class SessionCatalog( /** * List all matching temp views in the specified database, including global/local temporary views. */ - def listTempViews(db: String, pattern: String): Seq[TableIdentifier] = { + def listTempViews(db: String, pattern: String): Seq[CatalogTable] = { val dbName = format(db) val globalTempViews = if (dbName == globalTempViewManager.database) { - globalTempViewManager.listViewNames(pattern).map { name => - TableIdentifier(name, Some(globalTempViewManager.database)) + globalTempViewManager.listViewNames(pattern).map { viewName => + globalTempViewManager.get(viewName).map(_.tableMeta).getOrElse( + throw new NoSuchTableException(globalTempViewManager.database, viewName)) } } else { Seq.empty } - dbViews ++ listLocalTempViews(pattern) + val localTempViews = listLocalTempViews(pattern).map { viewIndent => + tempViews.get(viewIndent.table).map(_.tableMeta).getOrElse( + throw new NoSuchTableException(viewIndent.database.getOrElse(""), viewIndent.table)) + } + + globalTempViews ++ localTempViews } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 5b065bb2ab88..b44bb26b53c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -4059,19 +4059,25 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { */ override def visitShowTableExtended( ctx: ShowTableExtendedContext): LogicalPlan = withOrigin(ctx) { - def createUnresolvedTable( + @inline def createUnresolvedTable( nsCtx: IdentifierReferenceContext, patternCtx: StringLitContext): LogicalPlan = withOrigin(patternCtx) { - val ns = Option(nsCtx).map(x => visitMultipartIdentifier(x.multipartIdentifier())) - UnresolvedTable(ns.getOrElse(Seq.empty[String]) :+ string(visitStringLit(patternCtx)), - "SHOW TABLE EXTENDED ... PARTITION ...") + if (nsCtx != null) { + withIdentClause(nsCtx, ns => { + val names = ns :+ string(visitStringLit(patternCtx)) + UnresolvedTable(names, "SHOW TABLE EXTENDED ... PARTITION ...") + }) + } else { + val names = Seq.empty[String] :+ string(visitStringLit(patternCtx)) + UnresolvedTable(names, "SHOW TABLE EXTENDED ... PARTITION ...") + } } Option(ctx.partitionSpec).map { spec => val table = createUnresolvedTable(ctx.identifierReference(), ctx.pattern) ShowTablePartition(table, UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(spec))) }.getOrElse { val ns = if (ctx.identifierReference() != null) { - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)) + withIdentClause(ctx.identifierReference, UnresolvedNamespace) } else { UnresolvedNamespace(Seq.empty[String]) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala index 8c70e7a55e12..f3316ddd7eec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala @@ -25,14 +25,14 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal, ToPrettyString} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, StringUtils} import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, SupportsPartitionManagement, Table, TableCatalog} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits.TableHelper -import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.sql.types.StructType /** * Physical plan node for showing tables without partition, Show the information of tables. @@ -63,14 +63,12 @@ case class ShowTablesExtendedExec( case Seq(db) => Some(db) case _ => None } - val views = sessionCatalog.listTempViews(db.getOrElse(""), pattern) - views.map { viewIdent => - val database = viewIdent.database.getOrElse("") - val tableName = viewIdent.table - val isTemp = sessionCatalog.isTempView(viewIdent) - val view = sessionCatalog.getTempViewOrPermanentTableMetadata(viewIdent) - val information = view.simpleString - rows += toCatalystRow(database, tableName, isTemp, s"$information\n") + val tempViews = sessionCatalog.listTempViews(db.getOrElse(""), pattern) + tempViews.map { tempView => + val database = tempView.identifier.database.getOrElse("") + val tableName = tempView.identifier.table + val information = tempView.simpleString + rows += toCatalystRow(database, tableName, true, s"$information\n") } rows.toSeq @@ -106,7 +104,7 @@ case class ShowTablesExtendedExec( .sortBy(_._1).map { case (key, value) => key + "=" + value }.mkString("[", ",", "]") - if (table.properties().isEmpty) { + if (!table.properties().isEmpty) { results.put("Table Properties", properties.mkString("[", ", ", "]")) } @@ -162,14 +160,15 @@ case class ShowTablePartitionExec( if (partitionIdentifiers.isEmpty) { throw QueryCompilationErrors.notExistPartitionError(tableIdent, ident, partitionSchema) } + assert(partitionIdentifiers.length == 1) val row = partitionIdentifiers.head 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 = - Cast(Literal(row.get(i, dataType), dataType), StringType, Some(timeZoneId)).eval() + 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) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index 5672a06d2762..f9a88aa3ba2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -52,19 +52,18 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { protected def namespaceKey: String = "Database" - protected def extendedTableExpectedResultDiff: String + protected def extendedTableInfo: String private def extendedTableExpectedResult( catalog: String, - namespaceName: String, namespace: String, table: String, partColName: String, dataColName: String): String = { s"""Catalog: $catalog - |$namespaceName: $namespace + |$namespaceKey: $namespace |Table: $table - |$extendedTableExpectedResultDiff + |$extendedTableInfo |Partition Provider: Catalog |Partition Columns: [`$partColName`] |Schema: root @@ -293,7 +292,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { // replace "Created Time", "Last Access", "Created By", "Location" val actualResult_0_3 = replace(resultCollect(0)(3).toString) val expectedResult_0_3 = extendedTableExpectedResult( - catalog, namespaceKey, namespace, table, "id", "data") + catalog, namespace, table, "id", "data") assert(actualResult_0_3 === expectedResult_0_3) assert(resultCollect(1).length == 4) @@ -302,7 +301,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val actualResult_1_3 = replace(resultCollect(1)(3).toString) // replace "Table Properties" val expectedResult_1_3 = extendedTableExpectedResult( - catalog, namespaceKey, namespace, table1, "id1", "data1") + catalog, namespace, table1, "id1", "data1") assert(actualResult_1_3 === expectedResult_1_3) assert(resultCollect(2).length == 4) @@ -311,7 +310,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val actualResult_2_3 = replace(resultCollect(2)(3).toString) // replace "Table Properties" val expectedResult_2_3 = extendedTableExpectedResult( - catalog, namespaceKey, namespace, table2, "id2", "data2") + catalog, namespace, table2, "id2", "data2") assert(actualResult_2_3 === expectedResult_2_3) } } @@ -412,4 +411,55 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { case other => other }.mkString("\n") } + + /** + * - V1: `show extended` and `select *` display the schema, + * the partition columns will be always displayed at the end. + * - V2: `show extended` display the schema, the partition columns + * will be always displayed at the end, + * but `select *` will respect the original table schema. + */ + protected def selectCommandSchema: Seq[String] = Seq("data", "id") + test("show table extended: partition columns are always showed at the end") { + val namespace = "ns1" + val table = "tbl" + withNamespaceAndTable(namespace, table, catalog) { _ => + sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + s"$defaultUsing PARTITIONED BY (id)") + sql(s"INSERT INTO $catalog.$namespace.$table PARTITION (id = 1) (data) VALUES ('data1')") + val result = sql(s"SELECT * FROM $catalog.$namespace.$table") + assert(result.schema.fieldNames === selectCommandSchema) + + val table1 = "tbl1" + withTable(table1) { + sql(s"CREATE TABLE $catalog.$namespace.$table1 (data string, id bigint) " + + s"$defaultUsing PARTITIONED BY (id)") + sql(s"INSERT INTO $catalog.$namespace.$table1 PARTITION (id = 1) (data) VALUES ('data2')") + val result1 = sql(s"SELECT * FROM $catalog.$namespace.$table1") + assert(result1.schema.fieldNames === Seq("data", "id")) + + val extendedResult = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table*'"). + sort("tableName") + val extendedResultCollect = extendedResult.collect() + + assert(extendedResultCollect(0)(1) === table) + assert(extendedResultCollect(0)(3).toString.endsWith( + """Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true) + | + |""".stripMargin + )) + + assert(extendedResultCollect(1)(1) === table1) + assert(extendedResultCollect(1)(3).toString.endsWith( + """Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true) + | + |""".stripMargin + )) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala index a62f82515dc9..4b4742910bd1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala @@ -170,7 +170,7 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase { |Created Time: |Last Access: """.stripMargin - protected override def extendedTableExpectedResultDiff: String = + protected override def extendedTableInfo: String = """Created Time: |Last Access: |Created By: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index 90ad16c4a4d3..c1a86a6bde1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -59,8 +59,11 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase protected override def namespaceKey: String = "Namespace" - protected override def extendedTableExpectedResultDiff: String = + protected override def extendedTableInfo: String = s"""Type: MANAGED |Provider: _ - |Owner: ${Utils.getCurrentUserName()}""".stripMargin + |Owner: ${Utils.getCurrentUserName()} + |Table Properties:
""".stripMargin + + protected override def selectCommandSchema: Seq[String] = Seq("id", "data") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala index 1a7d9c4b4175..79b1eb6c0961 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala @@ -55,7 +55,7 @@ class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { |Created Time: |Last Access: """.stripMargin - protected override def extendedTableExpectedResultDiff: String = + protected override def extendedTableInfo: String = s"""Owner: ${Utils.getCurrentUserName()} |Created Time: |Last Access: From e6978e369cfba0d721e6e5758d57550b817c9c0b Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 6 Nov 2023 19:51:16 +0800 Subject: [PATCH 27/29] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 --- dev/appveyor-install-dependencies.ps1 | 2 - .../sql/catalyst/catalog/SessionCatalog.scala | 10 ++- .../sql/catalyst/parser/AstBuilder.scala | 23 +++---- .../v2/ShowTablesExtendedExec.scala | 3 +- .../command/ShowTablesSuiteBase.scala | 66 +++++++++---------- .../command/v2/ShowTablesSuite.scala | 7 +- 6 files changed, 52 insertions(+), 59 deletions(-) diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index d6382aaec019..f410e8774d96 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -27,13 +27,11 @@ Function InstallR { $urlPath = "" $latestVer = $(ConvertFrom-JSON $(Invoke-WebRequest https://rversions.r-pkg.org/r-release-win).Content).version - $latestVer = "4.3.2" If ($rVer -ne $latestVer) { $urlPath = ("old/" + $rVer + "/") } $rurl = $CRAN + "/bin/windows/base/" + $urlPath + "R-" + $rVer + "-win.exe" - echo $rurl # Downloading R Start-FileDownload $rurl "R-win.exe" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e53b29716325..28d35ab1045f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1096,17 +1096,15 @@ class SessionCatalog( def listTempViews(db: String, pattern: String): Seq[CatalogTable] = { val dbName = format(db) val globalTempViews = if (dbName == globalTempViewManager.database) { - globalTempViewManager.listViewNames(pattern).map { viewName => - globalTempViewManager.get(viewName).map(_.tableMeta).getOrElse( - throw new NoSuchTableException(globalTempViewManager.database, viewName)) + globalTempViewManager.listViewNames(pattern).flatMap { viewName => + globalTempViewManager.get(viewName).map(_.tableMeta) } } else { Seq.empty } - val localTempViews = listLocalTempViews(pattern).map { viewIndent => - tempViews.get(viewIndent.table).map(_.tableMeta).getOrElse( - throw new NoSuchTableException(viewIndent.database.getOrElse(""), viewIndent.table)) + val localTempViews = listLocalTempViews(pattern).flatMap { viewIndent => + tempViews.get(viewIndent.table).map(_.tableMeta) } globalTempViews ++ localTempViews diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index b44bb26b53c0..41c44da9a326 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -4059,21 +4059,18 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { */ override def visitShowTableExtended( ctx: ShowTableExtendedContext): LogicalPlan = withOrigin(ctx) { - @inline def createUnresolvedTable( - nsCtx: IdentifierReferenceContext, - patternCtx: StringLitContext): LogicalPlan = withOrigin(patternCtx) { - if (nsCtx != null) { - withIdentClause(nsCtx, ns => { - val names = ns :+ string(visitStringLit(patternCtx)) + Option(ctx.partitionSpec).map { spec => + val table = withOrigin(ctx.pattern) { + if (ctx.identifierReference() != null) { + withIdentClause(ctx.identifierReference(), ns => { + val names = ns :+ string(visitStringLit(ctx.pattern)) + UnresolvedTable(names, "SHOW TABLE EXTENDED ... PARTITION ...") + }) + } else { + val names = Seq.empty[String] :+ string(visitStringLit(ctx.pattern)) UnresolvedTable(names, "SHOW TABLE EXTENDED ... PARTITION ...") - }) - } else { - val names = Seq.empty[String] :+ string(visitStringLit(patternCtx)) - UnresolvedTable(names, "SHOW TABLE EXTENDED ... PARTITION ...") + } } - } - Option(ctx.partitionSpec).map { spec => - val table = createUnresolvedTable(ctx.identifierReference(), ctx.pattern) ShowTablePartition(table, UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(spec))) }.getOrElse { val ns = if (ctx.identifierReference() != null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala index f3316ddd7eec..da0a74f93e4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala @@ -118,8 +118,7 @@ case class ShowTablesExtendedExec( } if (table.schema().nonEmpty) { - val dataColumns = table.schema().filterNot(partitionColumns.contains) - results.put("Schema", StructType(dataColumns ++ partitionColumns).treeString) + results.put("Schema", table.schema().treeString) } results.map { case (key, value) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index f9a88aa3ba2d..c88217221ab7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -54,12 +54,17 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { protected def extendedTableInfo: String + protected def extendedTableSchema: String = + s"""Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin + private def extendedTableExpectedResult( catalog: String, namespace: String, table: String, - partColName: String, - dataColName: String): String = { + dataColName: String, + partColName: String): String = { s"""Catalog: $catalog |$namespaceKey: $namespace |Table: $table @@ -191,7 +196,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val namespace = "ns1" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing PARTITIONED BY (id)") + sql(s"CREATE TABLE $tbl (data string, id bigint) $defaultUsing PARTITIONED BY (id)") sql(s"ALTER TABLE $tbl ADD PARTITION (id = 1)") checkError( exception = intercept[AnalysisException] { @@ -210,7 +215,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val namespace = "ns1" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id bigint, data string) $defaultUsing") + sql(s"CREATE TABLE $tbl (data string, id bigint) $defaultUsing") val e = intercept[AnalysisException] { sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id = 1)") } @@ -224,7 +229,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val namespace = "ns1" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + + sql(s"CREATE TABLE $tbl (data string, id1 bigint, id2 bigint) " + s"$defaultUsing PARTITIONED BY (id1, id2)") sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") @@ -247,7 +252,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val namespace = "ns1" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { tbl => - sql(s"CREATE TABLE $tbl (id1 bigint, id2 bigint, data string) " + + sql(s"CREATE TABLE $tbl (data string, id1 bigint, id2 bigint) " + s"$defaultUsing PARTITIONED BY (id1, id2)") sql(s"ALTER TABLE $tbl ADD PARTITION (id1 = 1, id2 = 2)") @@ -269,14 +274,14 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val namespace = "ns1" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { _ => - sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + sql(s"CREATE TABLE $catalog.$namespace.$table (data string, id bigint) " + s"$defaultUsing PARTITIONED BY (id)") val table1 = "tbl1" val table2 = "tbl2" withTable(table1, table2) { - sql(s"CREATE TABLE $catalog.$namespace.$table1 (id1 bigint, data1 string) " + + sql(s"CREATE TABLE $catalog.$namespace.$table1 (data1 string, id1 bigint) " + s"$defaultUsing PARTITIONED BY (id1)") - sql(s"CREATE TABLE $catalog.$namespace.$table2 (id2 bigint, data2 string) " + + sql(s"CREATE TABLE $catalog.$namespace.$table2 (data2 string, id2 bigint) " + s"$defaultUsing PARTITIONED BY (id2)") val result = sql(s"SHOW TABLE EXTENDED FROM $catalog.$namespace LIKE '$table*'") @@ -292,7 +297,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { // replace "Created Time", "Last Access", "Created By", "Location" val actualResult_0_3 = replace(resultCollect(0)(3).toString) val expectedResult_0_3 = extendedTableExpectedResult( - catalog, namespace, table, "id", "data") + catalog, namespace, table, "data", "id") assert(actualResult_0_3 === expectedResult_0_3) assert(resultCollect(1).length == 4) @@ -301,7 +306,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val actualResult_1_3 = replace(resultCollect(1)(3).toString) // replace "Table Properties" val expectedResult_1_3 = extendedTableExpectedResult( - catalog, namespace, table1, "id1", "data1") + catalog, namespace, table1, "data1", "id1") assert(actualResult_1_3 === expectedResult_1_3) assert(resultCollect(2).length == 4) @@ -310,7 +315,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { val actualResult_2_3 = replace(resultCollect(2)(3).toString) // replace "Table Properties" val expectedResult_2_3 = extendedTableExpectedResult( - catalog, namespace, table2, "id2", "data2") + catalog, namespace, table2, "data2", "id2") assert(actualResult_2_3 === expectedResult_2_3) } } @@ -415,50 +420,41 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { /** * - V1: `show extended` and `select *` display the schema, * the partition columns will be always displayed at the end. - * - V2: `show extended` display the schema, the partition columns - * will be always displayed at the end, - * but `select *` will respect the original table schema. + * - V2: `show extended` and `select *` display the schema, + * the columns order will respect the original table schema. */ - protected def selectCommandSchema: Seq[String] = Seq("data", "id") - test("show table extended: partition columns are always showed at the end") { + protected def selectCommandSchema: Array[String] = Array("data", "id") + test("show table extended: the display order of the columns is different in v1 and v2") { val namespace = "ns1" val table = "tbl" withNamespaceAndTable(namespace, table, catalog) { _ => - sql(s"CREATE TABLE $catalog.$namespace.$table (id bigint, data string) " + + sql(s"CREATE TABLE $catalog.$namespace.$table (data string, id bigint) " + s"$defaultUsing PARTITIONED BY (id)") sql(s"INSERT INTO $catalog.$namespace.$table PARTITION (id = 1) (data) VALUES ('data1')") val result = sql(s"SELECT * FROM $catalog.$namespace.$table") - assert(result.schema.fieldNames === selectCommandSchema) + assert(result.schema.fieldNames === Array("data", "id")) val table1 = "tbl1" withTable(table1) { - sql(s"CREATE TABLE $catalog.$namespace.$table1 (data string, id bigint) " + + sql(s"CREATE TABLE $catalog.$namespace.$table1 (id bigint, data string) " + s"$defaultUsing PARTITIONED BY (id)") sql(s"INSERT INTO $catalog.$namespace.$table1 PARTITION (id = 1) (data) VALUES ('data2')") + val result1 = sql(s"SELECT * FROM $catalog.$namespace.$table1") - assert(result1.schema.fieldNames === Seq("data", "id")) + assert(result1.schema.fieldNames === selectCommandSchema) val extendedResult = sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table*'"). sort("tableName") val extendedResultCollect = extendedResult.collect() assert(extendedResultCollect(0)(1) === table) - assert(extendedResultCollect(0)(3).toString.endsWith( - """Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true) - | - |""".stripMargin - )) + assert(extendedResultCollect(0)(3).toString.contains( + s"""Schema: root + | |-- data: string (nullable = true) + | |-- id: long (nullable = true)""".stripMargin)) assert(extendedResultCollect(1)(1) === table1) - assert(extendedResultCollect(1)(3).toString.endsWith( - """Schema: root - | |-- data: string (nullable = true) - | |-- id: long (nullable = true) - | - |""".stripMargin - )) + assert(extendedResultCollect(1)(3).toString.contains(extendedTableSchema)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala index c1a86a6bde1d..d66dca20d77b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala @@ -65,5 +65,10 @@ class ShowTablesSuite extends command.ShowTablesSuiteBase with CommandSuiteBase |Owner: ${Utils.getCurrentUserName()} |Table Properties:
""".stripMargin - protected override def selectCommandSchema: Seq[String] = Seq("id", "data") + protected override def extendedTableSchema: String = + s"""Schema: root + | |-- id: long (nullable = true) + | |-- data: string (nullable = true)""".stripMargin + + protected override def selectCommandSchema: Array[String] = Array("id", "data") } From 26ae6795024261c9d261b2bde13f9518c075d913 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 7 Nov 2023 13:59:47 +0800 Subject: [PATCH 28/29] fix minor --- .../org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 28d35ab1045f..e9a02a243aa5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -1094,8 +1094,7 @@ class SessionCatalog( * List all matching temp views in the specified database, including global/local temporary views. */ def listTempViews(db: String, pattern: String): Seq[CatalogTable] = { - val dbName = format(db) - val globalTempViews = if (dbName == globalTempViewManager.database) { + val globalTempViews = if (format(db) == globalTempViewManager.database) { globalTempViewManager.listViewNames(pattern).flatMap { viewName => globalTempViewManager.get(viewName).map(_.tableMeta) } From d318f6bf93553de76fdff61d687806b5dc9251ef Mon Sep 17 00:00:00 2001 From: panbingkun Date: Sun, 12 Nov 2023 21:03:37 +0800 Subject: [PATCH 29/29] Trigger build