You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/08/20 11:41:42 UTC

[GitHub] [spark] panbingkun opened a new pull request, #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

panbingkun opened a new pull request, #37588:
URL: https://github.com/apache/spark/pull/37588

   ### What changes were proposed in this pull request?
   The pr aim to implement v2 SHOW TABLE EXTENDED as `ShowTableExec`
   
   ### Why are the changes needed?
   To have feature parity with the datasource V1.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, Support SHOW TABLE EXTENDED in v2.
   
   ### How was this patch tested?
   By running the unified tests for v2 implementation:
   ```
   $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowTablesSuite"
   $ build/sbt "test:testOnly *ShowTablesSuite"
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1108213058


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   `TablePartitionSpec` is legacy one, can't you use `ResolvedPartitionSpec`? For example, see https://github.com/apache/spark/blob/0494dc90af48ce7da0625485a4dc6917a244d580/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowPartitionsExec.scala#L36



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +87,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(

Review Comment:
   This one is not needed. The job should be done by `ResolvePartitionSpec.resolvePartitionSpec`, or there is some reason to bypass it?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +87,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 =>

Review Comment:
   The functions `extendedPartition()` is invoked only for non-empty partition spec as I can see, or not? Is there any test for this case?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +87,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) =>

Review Comment:
   nit:
   ```suggestion
         case len if len > 1 =>
   ```



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +87,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

Review Comment:
   Where is it used?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +87,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) {

Review Comment:
   This loop:
   ```scala
       var i = 0
       while (i < len) {
   
         i += 1
       }
   ```
   can be simplified by:
   ```scala
       for (i <- 0 until len) {
   
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1109493252


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   In fact, `ResolvedPartitionSpec` cannot be used here, because the table(`Pattern` match may have multiple tables) has not been determined.
   The scene is different from `ShowPartitionExec`.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   @MaxGekk
   two chooses:
   1.AstBuilder(visitShowTableExtended) don't use UnresolvedPartitionSpec, instead use Map<String, String>. Because of the lack of `decisive table`, it is impossible to resolve to `ResolvedPartitionSpec`.
   2.AstBuilder(visitShowTableExtended) use UnresolvedPartitionSpec, have to add a subclass(eg: ExtractPartitionSpec, it's `resolved` is true)
   ### The root cause is that the table is `non-deterministic` (Pattern match may have multiple tables) in the ShowTableExtended.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1769744166

   In addition to the issue of replacing strings with variable names in the review, I also found several other similar points in `ShowTablesSuite` that have been corrected.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381654280


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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)

Review Comment:
   When a partition does not exist, it triggers this exception
   Because we didn't actually retrieve this partition from the meta during `UnresolvedPartitionSpec` to `ResolvedPartitionSpec`.
   https://github.com/apache/spark/blob/1359c1327345efdf9a35c46a355b5f928ac33e6d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala#L42-L74
   
   According to my understanding, it only detects whether the `partition key` is valid and does some normalized of the `partition key`, the existence of the `partition value` is not verified.
   If my understanding is incorrect, please point it out.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1813824544

   > thanks, merging to master!
   
   Thank you again for your great help! ❤️❤️❤️


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375783886


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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 - " +

Review Comment:
   what's the v2 difference for this test?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375752983


##########
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(

Review Comment:
   `if (StringUtils.filterPattern(Seq(...), pattern).nonEmpty)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374586437


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -2785,4 +2785,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
         "upper" -> size.toString,
         "invalidValue" -> pos.toString))
   }
+
+  def notExistPartitionError(

Review Comment:
   shall we put it in `QueryCompilationErrors`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376065102


##########
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)

Review Comment:
   My first version used `Table#schema` directly, but during UT testing, it was found that the schema field information partition field was not displayed at the end, which is different from the result of the `V1 command`. Do we want to maintain consistency? Or this order is actually not important.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1109667336


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   @MaxGekk
   two chooses:
   1.AstBuilder(visitShowTableExtended) don't use UnresolvedPartitionSpec, instead use Map<String, String>. Because of the lack of `decisive table`, it is impossible to resolve to `ResolvedPartitionSpec`.
   2.AstBuilder(visitShowTableExtended) use UnresolvedPartitionSpec, have to add a subclass(eg: ExtractPartitionSpec, it's `resolved` is true)
   ### The root cause is that the table is `non-deterministic` (Pattern match may have multiple tables) in the ShowTableExtended.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1109667336


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   two chooses:
   1.AstBuilder(visitShowTableExtended) don't use UnresolvedPartitionSpec, instead, use Map<String, String>. Because of the lack of `decisive table`, it is impossible to resolve to `ResolvedPartitionSpec`.
   2.AstBuilder(visitShowTableExtended) use UnresolvedPartitionSpec, have to add a subclass(eg: ExtractPartitionSpec, it's `resolved` is true)
   The root cause is that the table is `non-deterministic` (Pattern match may have multiple tables) in the ShowTableExtended.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1365547706


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,15 +41,36 @@ 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[ResolvedPartitionSpec] = 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))
+    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)

Review Comment:
   seems we should not load the table if it's a temp view?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1365553647


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -18,8 +18,9 @@
 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.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.

Review Comment:
   is there any test case we can move to the base test suite now? Otherwise it's hard to track the behavior difference between v1 and v2 command.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1367896086


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,15 +41,36 @@ 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[ResolvedPartitionSpec] = 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))
+    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)

Review Comment:
   From the current logic, it seems that if the catalog is V2SessionCatalog, it can be loaded, although we can see from the following code comments that a `NoSuchTableException` should be thrown at this point.
   https://github.com/apache/spark/blob/2709426f0f622a214c51954664458e7dd2ab3304/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableCatalog.java#L103-L104
   If my understanding is incorrect, please correct me.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1372537061


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -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)

Review Comment:
   Okay, Let's separate it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374599990


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,17 +41,36 @@ 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[ResolvedPartitionSpec] = 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))
+    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)

Review Comment:
   this looks wrong to me. According to the classdoc of `TableCatalog.listTables`, it should not return views.
   
   Since temp views are managed fully by Spark, we can just invoke internal Spark APIs to list temp views, and then call `TableCatalog.listTables`, and finally combine the results.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1379487429


##########
dev/appveyor-install-dependencies.ps1:
##########
@@ -27,11 +27,13 @@ Function InstallR {
 
   $urlPath = ""
   $latestVer = $(ConvertFrom-JSON $(Invoke-WebRequest https://rversions.r-pkg.org/r-release-win).Content).version
+  $latestVer = "4.3.2"

Review Comment:
   This modification is not related to this PR, but is caused by another issue: https://github.com/apache/spark/pull/43631. Only in order to temporarily pass `integration appveyor`, I will remove it after it is completed.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375431259


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2133,12 +2133,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
         "inputTypesLen" -> bound.inputTypes().length.toString))
   }
 
-  def commandUnsupportedInV2TableError(name: String): Throwable = {
-    new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1200",
-      messageParameters = Map("name" -> name))
-  }

Review Comment:
   Yes, I misunderstood your meaning. That's right, we can remove it from `error-classes. json`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380268144


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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()

Review Comment:
   `ToPrettyString` is probably better here, which is used by `df.show`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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()

Review Comment:
   `ToPrettyString` is probably better here, which is used by `df.show`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380269913


##########
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] = {

Review Comment:
   shall we just return `Seq[CatalogTable]`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1384419375


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -1090,6 +1090,26 @@ 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[CatalogTable] = {
+    val dbName = format(db)
+    val globalTempViews = if (dbName == globalTempViewManager.database) {

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r956715662


##########
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

Review Comment:
   resolved = true



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1416896194

   > @panbingkun Could you resolve conflicts when you have time, please.
   
   Done


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1109493252


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   In fact, `ResolvedPartitionSpec` cannot be used here, because the table(`Pattern` match may have multiple tables) has not been determined.
   The scene is different from `ShowPartitionExec`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1477419783

   @MaxGekk It's appreciated if it can be reviewed in your convenience, thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1389120888


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,17 +41,36 @@ 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[ResolvedPartitionSpec] = 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))
+    if (partitionSpec.isEmpty) {
+      // Show the information of tables.
+      val identifiers = catalog.listTables(namespace.toArray)

Review Comment:
   @cloud-fan 
   Based on the above suggestions, a separate PR has been submitted to implement `Identifier[] listTables(String[] namespace, String pattern)` in `TableCatalog`.
   https://github.com/apache/spark/pull/43751



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380290514


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -126,4 +158,258 @@ 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 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 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"
+    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)")
+
+      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"))
+        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"
+    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 = replace(localResultCollect(0)(3).toString)
+        val expectedLocalResult =
+          s"""Table: $localTmpViewName
+             |Created Time: <created time>
+             |Last Access: <last access>
+             |Created By: <created by>
+             |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*'").

Review Comment:
   ```suggestion
           val globalResult = sql(s"SHOW TABLE EXTENDED IN global_temp LIKE '$viewName*'").
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380291630


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,63 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase {
       }
     }
   }
+
+  override protected def extendedPartInNonPartedTableError(
+      catalog: String,
+      namespace: String,
+      table: String): (String, Map[String, String]) = {
+    ("_LEGACY_ERROR_TEMP_1251",

Review Comment:
   how hard it is to unify this error between v1 and v2 tables?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375784149


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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"))
+
+      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") {

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1378345147


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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 - " +

Review Comment:
   can we have some way to abstract it out? e.g. the sub test suite can add some custom information in the result.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1378705860


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   @cloud-fan 
   Do we really need to split a UT logic into subclasses in order not to `modify the above logic`? Separate into two subclasses, so it seems that the similarities cannot be seen anymore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1382826777


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -1090,6 +1090,28 @@ 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[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))

Review Comment:
   It's weird to throw this error during listing views. Shall we use `flatMap` and just skip the temp views that were deleted immediately after `globalTempViewManager.listViewNames`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1383248874


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -1090,6 +1090,26 @@ 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[CatalogTable] = {
+    val dbName = format(db)
+    val globalTempViews = if (dbName == globalTempViewManager.database) {

Review Comment:
   ```suggestion
       val globalTempViews = if (format(db) == globalTempViewManager.database) {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r956704088


##########
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],

Review Comment:
   Ok, done!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1111993130


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -2791,4 +2791,17 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
         "location" -> toSQLValue(location.toString, StringType),
         "identifier" -> toSQLId(tableId.nameParts)))
   }
+
+  def showTableExtendedMultiPartitionUnsupportedError(tableName: String): Throwable = {

Review Comment:
   I will refactor it later



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374601778


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +80,103 @@ case class ShowTablesExec(
       case _ => false
     }
   }
+
+  private def extendedTable(

Review Comment:
   the name looks weird, how about `getTableDetails`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375785577


##########
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 = {

Review Comment:
   I think `withTempView("global_temp.view_name")` should work?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1378438582


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   Let's avoid unrelated change I said.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1378350993


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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 - " +

Review Comment:
   Okay, I'll give it a try.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380218989


##########
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) {

Review Comment:
   ```suggestion
       val globalTempViews = if (dbName == globalTempViewManager.database) {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381654555


##########
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 = withOrigin(patternCtx) {
+      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)))

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381686442


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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)

Review Comment:
   ah I see!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380278496


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -126,4 +158,258 @@ 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") {

Review Comment:
   ```suggestion
     test("show table extended with no matching table") {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381658801


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -126,4 +158,258 @@ 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 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 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)")

Review Comment:
   I put this in `base suite` and compared the differences in the `order` of `data columns` and `partition columns` when displaying the schema of a table using "select *" and "show table extended...".



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381659193


##########
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

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1807069942

   unfortunately this has conflicts now...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1763551022

   We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
   If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1764531574

   @panbingkun Could you rebase it on the recent master, please.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1221421651

   cc @MaxGekk 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rshanmugam1 commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
rshanmugam1 commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1335963542

   +1


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Fokko commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
Fokko commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1329178849

   @panbingkun @MaxGekk Any progress on this? Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1611085292

   @panbingkun Please, rebase it on the recent master.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1382828588


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -1090,6 +1090,28 @@ 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[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))

Review Comment:
   Okay



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1382875485


##########
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)

Review Comment:
   OMG, I really missed it, let me fix it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1390118872


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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, 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.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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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(

Review Comment:
   ```suggestion
         results.put("Partition Columns", partitionColumns.map(
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1771049872

   Okay, I'll update later, Thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1377425633


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   I got it now. In fact, you added the test cases into the base class ShowTablesSuiteBase, so the output is different from v1 and v2.
   So we can add this test case into `v1.ShowTablesSuite` and `v2.ShowTablesSuite` and avoid add `toSQLId`.
   
   After this PR, you can create another PR to fix the error class.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   I got it now. In fact, you added the test cases into the base class `ShowTablesSuiteBase`, so the output is different from v1 and v2.
   So we can add this test case into `v1.ShowTablesSuite` and `v2.ShowTablesSuite` and avoid add `toSQLId`.
   
   After this PR, you can create another PR to fix the error class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380266348


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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

Review Comment:
   let's add an assert before calling head: `assert(partitionIdentifiers.length == 1)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380255218


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    val views = sessionCatalog.listTempViews(db.get, pattern)

Review Comment:
   ```suggestion
       val views = sessionCatalog.listTempViews(db.getOrElse(""), pattern)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375242759


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala:
##########
@@ -30,6 +30,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
 
 object DataSourceV2Implicits {
   implicit class TableHelper(table: Table) {
+

Review Comment:
   Okay



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375347289


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2133,12 +2133,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
         "inputTypesLen" -> bound.inputTypes().length.toString))
   }
 
-  def commandUnsupportedInV2TableError(name: String): Throwable = {
-    new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1200",
-      messageParameters = Map("name" -> name))
-  }

Review Comment:
   In fact, just remove the error class, it will not increase the complexity.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375243205


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +80,103 @@ case class ShowTablesExec(
       case _ => false
     }
   }
+
+  private def extendedTable(

Review Comment:
   Okay



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,17 +41,36 @@ 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[ResolvedPartitionSpec] = 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))
+    if (partitionSpec.isEmpty) {
+      // Show the information of tables.
+      val identifiers = catalog.listTables(namespace.toArray)

Review Comment:
   Okay



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1372536605


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -50,6 +50,16 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] {
           }
         case _ => command
       }
+    case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace),

Review Comment:
   According to my understanding of the processing logic of V2PartitionCommand in ResolvePartitionSpec, 
   the table it processes is a single table,
   
   However, our ShowTableExtended supports two situations:
   - (A).SHOW TABLE EXTENDED FROM/IN catalog.namespace LIKE 'table_name_pattern'; --- support a table regex pattern 
   - (B).SHOW TABLE EXTENDED FROM/IN catalog.namespace LIKE 'table_name' PARTITION (part1 = 1); --- not support a table regex pattern, only support a certain table name.
   https://docs.databricks.com/en/sql/language-manual/sql-ref-syntax-aux-show-table.html
   <img width="738" alt="image" src="https://github.com/apache/spark/assets/15246973/95e62919-1777-4421-a759-c84626782e2b">
   
   In case A, the tables it processes may be multiple tables, which is a regex pattern. It seems inappropriate to resolve the table in `ResolvePartitionSpec`.
   In case B, the tables it processes is single (currently we only support the partitioned extended display information function in the case of one table)
   
   If it is only a table name and partition information is specified, we can resolved partition here (covering situation B)
   so made a special logic here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1372583615


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -50,6 +50,16 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] {
           }
         case _ => command
       }
+    case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace),

Review Comment:
   I think we need two commands for this SQL statement: `ShowTablesExtended` and `ShowTablePartition`. In `ShowTablePartition`, we can put `UnresolvedTable` in it and extend `V2PartitionCommand`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375746540


##########
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(

Review Comment:
   Let's not rename the parser rule now. It matches both commands.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375760595


##########
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) {

Review Comment:
   I think we should always put a namespace field, empty string if it's Nil



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376099810


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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 - " +

Review Comment:
   The difference between the results of `v1` and `v2` is not significant, mainly due to the addition of information in the red box below in the `hive` results
   <img width="614" alt="image" src="https://github.com/apache/spark/assets/15246973/046b2032-cd0b-4a23-9d82-5b87c42774c5">
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r956715554


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -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))
   }

Review Comment:
   UnresolvedPartitionSpec to ExtractPartitionSpec
   The difference between the two is that the former is unresolved and the latter is resolved



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1109667336


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   @MaxGekk
   two chooses:
   1.AstBuilder(visitShowTableExtended) don't use UnresolvedPartitionSpec, instead use Map<String, String>. Because of the lack of `decisive table`, it is impossible to resolve to `ResolvedPartitionSpec`.
   2.AstBuilder(visitShowTableExtended) use UnresolvedPartitionSpec, have to add a subclass(eg: ExtractPartitionSpec, it's `resolved` is true)
   The root cause is that the table is `non-deterministic` (Pattern match may have multiple tables) in the ShowTableExtended.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1612349461

   > @panbingkun Please, rebase it on the recent master.
   
   Done.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1813802035

   thanks, merging to master!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376075950


##########
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 = {

Review Comment:
   If written like that, then in the code we have to write like this
   <img width="736" alt="image" src="https://github.com/apache/spark/assets/15246973/d99021b5-ad00-450d-8ee2-7152c626c430">
   
   <img width="896" alt="image" src="https://github.com/apache/spark/assets/15246973/ca816b66-24e1-49e4-b83e-9256810952cf">
   
   https://github.com/apache/spark/blob/e499ba80be6828dcf0c34d3f06f1cd12321ce5b8/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala#L550-L554



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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"))
+
+      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)
+      }
+    }
+  }
+
+  test("show table extended in view: permanent, temp global, temp local") {

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376105978


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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"))
+
+      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") {

Review Comment:
   - v1
      <img width="414" alt="image" src="https://github.com/apache/spark/assets/15246973/ad7c2dbb-b843-4ece-8b07-cc465fac94f4">
   
   - v2
      <img width="404" alt="image" src="https://github.com/apache/spark/assets/15246973/2ac18418-050d-4cc0-8997-b9879bd2e96b">
   
   - hive
      <img width="601" alt="image" src="https://github.com/apache/spark/assets/15246973/75548e6f-889c-4f2b-8c44-cd33496ed3bc">



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1377094224


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   Hmm... this is a newly added UT. 😄



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   Hmm... this is a newly added UT. 😄



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380261145


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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) {

Review Comment:
   do you mean `properties.nonEmpty`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380278877


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -126,4 +158,258 @@ 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 a not existing partition") {

Review Comment:
   ```suggestion
     test("show table extended with a not existing partition") {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1791622292

   > I think it's pretty close now, thanks for your patience!
   
   I will update it again today. 
   Thank you very much for your patience and seriousness, which has been a great help to me!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380268928


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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()

Review Comment:
   for example, null value is handled well by it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380247392


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   This change is necessary as it helps to unify the v1/v2 command behavior, which is an important goal of adding new v2 commands



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381660206


##########
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,

Review Comment:
   Yes, it was my bad.   I have corrected it.



##########
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] = {

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1382830431


##########
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)

Review Comment:
   @panbingkun Seems this is missed. We should use the original table schema here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1367896135


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +82,95 @@ 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())

Review Comment:
   Yes



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala:
##########
@@ -65,6 +70,11 @@ object DataSourceV2Implicits {
       }
     }
 
+    def isPartitionable: Boolean = table match {

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1367896387


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -18,8 +18,9 @@
 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.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.

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2
URL: https://github.com/apache/spark/pull/37588


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1365548828


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +82,95 @@ 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())

Review Comment:
   is it to follow the v1 behavior?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1365540128


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala:
##########
@@ -65,6 +70,11 @@ object DataSourceV2Implicits {
       }
     }
 
+    def isPartitionable: Boolean = table match {

Review Comment:
   `def supportsPartitions`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1111839300


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] ja-michel commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
ja-michel commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1331815161

   +1


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1411562482

   @panbingkun Could you resolve conflicts when you have time, please.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1372606005


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -50,6 +50,16 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] {
           }
         case _ => command
       }
+    case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace),

Review Comment:
   Ok, I think it's also fine, Let me update it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1367896333


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +82,95 @@ 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
+    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.isReadable) {
+      if (table.schema().nonEmpty) results.put("Schema", table.schema().treeString)

Review Comment:
   Okay, Let's remove `if (table.isReadable)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1372811304


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -50,6 +50,16 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] {
           }
         case _ => command
       }
+    case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace),

Review Comment:
   Resolve ShowTableExtended here, looks weird.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374595749


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala:
##########
@@ -404,6 +404,17 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case ShowTables(ResolvedNamespace(catalog, ns), pattern, output) =>
       ShowTablesExec(output, catalog.asTableCatalog, ns, pattern) :: Nil
 
+    case ShowTablesExtended(
+        ResolvedNamespace(catalog, ns),
+        pattern,
+        output) =>
+      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()),

Review Comment:
   shall we also have two physical plans?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1807121343

   > unfortunately this has conflicts now...
   
   Done, I have resolved these conflicts.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1377086123


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   It seems there are two `invalidPartitionSpecError` with different signature.
   Before this PR, why `org.apache.spark.sql.execution.command.v1.ShowTablesSuite.show table extended in multi partition key - the command's partition parameters are incomplete` not failed?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375761643


##########
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)

Review Comment:
   shouldn't we use `Table#schema`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan closed pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2
URL: https://github.com/apache/spark/pull/37588


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1813555489

   @cloud-fan If you have time, could you please take a look at this PR?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1382826928


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:
##########
@@ -1090,6 +1090,28 @@ 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[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))
+      }
+    } else {
+      Seq.empty
+    }
+
+    val localTempViews = listLocalTempViews(pattern).map { viewIndent =>
+      tempViews.get(viewIndent.table).map(_.tableMeta).getOrElse(
+        throw new NoSuchTableException(viewIndent.database.getOrElse(""), viewIndent.table))

Review Comment:
   ditto



##########
dev/appveyor-install-dependencies.ps1:
##########
@@ -27,11 +27,13 @@ Function InstallR {
 
   $urlPath = ""
   $latestVer = $(ConvertFrom-JSON $(Invoke-WebRequest https://rversions.r-pkg.org/r-release-win).Content).version
+  $latestVer = "4.3.2"

Review Comment:
   We can remove it now?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381645724


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,63 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase {
       }
     }
   }
+
+  override protected def extendedPartInNonPartedTableError(
+      catalog: String,
+      namespace: String,
+      table: String): (String, Map[String, String]) = {
+    ("_LEGACY_ERROR_TEMP_1251",

Review Comment:
   - _LEGACY_ERROR_TEMP_1231(QueryCompilationErrors#invalidPartitionColumnKeyInTableError), there are approximately `13 `code point to call it:
      <img width="465" alt="image" src="https://github.com/apache/spark/assets/15246973/7d61f88a-c567-43a5-b7a9-5751ac61613d">
   https://github.com/apache/spark/blob/1359c1327345efdf9a35c46a355b5f928ac33e6d/common/utils/src/main/resources/error/error-classes.json#L4815-L4819
   
   - _LEGACY_ERROR_TEMP_1251(QueryCompilationErrors#actionNotAllowedOnTableSincePartitionMetadataNotStoredError), there are approximately `8 `code point to call it:
      <img width="711" alt="image" src="https://github.com/apache/spark/assets/15246973/0a18def1-b604-44ee-990f-ed1fdaa2d7a0">
   https://github.com/apache/spark/blob/1359c1327345efdf9a35c46a355b5f928ac33e6d/common/utils/src/main/resources/error/error-classes.json#L4860-L4864
   
   Because there are many scopes involved, in order to reduce the interference of logic on this PR, I suggest doing `the merging or unification` in a new separate PR. 
   Do you think this is appropriate?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380287495


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -126,4 +158,258 @@ 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 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 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"
+    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)")
+
+      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"))
+        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") {

Review Comment:
   ```suggestion
     test("show table extended with temp views") {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380283326


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -126,4 +158,258 @@ 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 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 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)")

Review Comment:
   nit: I think tests in the base suite should always partition by the ending columns. We can add a new simple test in v1 suite to prove that partition columns are always at the end, and a new simple test in v2 to prove that we respect the original table schema.
   
   I think the v1 behavior is probably a bug, but we may never fix it as it becomes a feature :(



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r951640493


##########
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],

Review Comment:
   Please, use the type `PartitionSpec`. The `TablePartitionSpec` is for v1 commands, and shouldn't be used in v2 commands.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1221489078

   Can one of the admins verify this patch?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1111993130


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -2791,4 +2791,17 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase {
         "location" -> toSQLValue(location.toString, StringType),
         "identifier" -> toSQLId(tableId.nameParts)))
   }
+
+  def showTableExtendedMultiPartitionUnsupportedError(tableName: String): Throwable = {

Review Comment:
   I will refactor it later



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375781992


##########
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)

Review Comment:
   so `partitionIdentifiers` must be size of 1?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376060024


##########
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(

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376219791


##########
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)

Review Comment:
   Yes. Shall we output the partition field even if it is empty?



##########
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)

Review Comment:
   I think we should keep consistent with V1.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375220594


##########
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)

Review Comment:
   It seems we can keep the line as
   `override val output: Seq[Attribute] = ShowTableExtended.getOutputAttrs) extends UnaryCommand {`



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala:
##########
@@ -30,6 +30,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
 
 object DataSourceV2Implicits {
   implicit class TableHelper(table: Table) {
+

Review Comment:
   No need.



##########
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(

Review Comment:
   Because we change `ShowTableExtended` to `ShowTablesExtended`, shall we update the name in g4 ?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2133,12 +2133,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
         "inputTypesLen" -> bound.inputTypes().length.toString))
   }
 
-  def commandUnsupportedInV2TableError(name: String): Throwable = {
-    new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1200",
-      messageParameters = Map("name" -> name))
-  }

Review Comment:
   We can remove the error class too.



##########
sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala:
##########
@@ -240,19 +240,33 @@ 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)
         output.head.withName("database") +: output.tail
       } 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)

Review Comment:
   I think this assert is no need.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2471,7 +2465,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       errorClass = "_LEGACY_ERROR_TEMP_1231",
       messageParameters = Map(
         "key" -> key,
-        "tblName" -> tblName))
+        "tblName" -> toSQLId(tblName)))

Review Comment:
   We can change it if we rename the error class.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   ditto



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2471,7 +2465,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       errorClass = "_LEGACY_ERROR_TEMP_1231",
       messageParameters = Map(
         "key" -> key,
-        "tblName" -> tblName))
+        "tblName" -> toSQLId(tblName)))

Review Comment:
   It seems this change is not related to PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376060396


##########
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) {

Review Comment:
   Okay



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1372811304


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -50,6 +50,16 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] {
           }
         case _ => command
       }
+    case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace),

Review Comment:
   Resolve `ShowTableExtended` here, looks weird.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374584553


##########
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 {

Review Comment:
   maybe name it `ShowTablesUtils` and remove `object ShowTablePartition`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374587818


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -2785,4 +2785,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
         "upper" -> size.toString,
         "invalidValue" -> pos.toString))
   }
+
+  def notExistPartitionError(

Review Comment:
   And it's hard to believe we don't have a function to throw `NoSuchPartitionException` already.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380276558


##########
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

Review Comment:
   this name is a bit confusing. Do you mean `extendedTableInfo`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381660558


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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()

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380265564


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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)

Review Comment:
   This can only happen when the partition is deleted right after we resolve to `ResolvedPartitionSpec`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1382827489


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4055,19 +4055,34 @@ 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)
+    @inline def createUnresolvedTable(

Review Comment:
   not this kind of inline... we can remove this function and put the code in where we call the function



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1383222640


##########
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)

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1767530944

   > @panbingkun Could you rebase it on the recent master, please.
   
   Done.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1109667336


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,14 +44,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 {

Review Comment:
   @MaxGekk
   two chooses:
   1.AstBuilder(visitShowTableExtended) don't use UnresolvedPartitionSpec, instead, use Map<String, String>. Because of the lack of `decisive table`, it is impossible to resolve to `ResolvedPartitionSpec`.
   2.AstBuilder(visitShowTableExtended) use UnresolvedPartitionSpec, have to add a subclass(eg: ExtractPartitionSpec, it's `resolved` is true)
   The root cause is that the table is `non-deterministic` (Pattern match may have multiple tables) in the ShowTableExtended.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1111841040


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +87,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(

Review Comment:
   Yes, the latest version has eliminated the above logic.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +87,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

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "xkrogen (via GitHub)" <gi...@apache.org>.
xkrogen commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1428236399

   @MaxGekk are you able to take another look at this PR? Would love to see it go in, this is an annoying feature gap that we need to work around currently for DSv2 sources :(


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381687816


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,63 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase {
       }
     }
   }
+
+  override protected def extendedPartInNonPartedTableError(
+      catalog: String,
+      namespace: String,
+      table: String): (String, Map[String, String]) = {
+    ("_LEGACY_ERROR_TEMP_1251",

Review Comment:
   make senses, let's do it in followup



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1383227023


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -126,4 +158,258 @@ 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 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 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)")

Review Comment:
   ` I think tests in the base suite should always partition by the ending columns. `
   I have adjusted the columns order of the `CREATE TABLE` in the `base suite` to end with partition columns.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380241060


##########
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 = withOrigin(patternCtx) {
+      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)))

Review Comment:
   or wrap the `UnresolvedTable`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380263403


##########
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)

Review Comment:
   I think we should respect the actual table schema. We can add one more abstraction in the base test suite to allow this difference.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380274949


##########
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,

Review Comment:
   does this need to be a parameter? isn't it always the value of `def namespaceKey`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375756682


##########
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

Review Comment:
   How can we include views if the given catalog is not a session catalog? I think we can only include temp views for now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375784614


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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"))
+
+      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)
+      }
+    }
+  }
+
+  test("show table extended in view: permanent, temp global, temp local") {

Review Comment:
   we should have separated tests for temp and permanent views. The v2 command should be able to handle temp views.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375783237


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase {
       }
     }
   }
+
+  test("show table extended in non-partitioned table") {

Review Comment:
   shouldn't v2 have the same error?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375244098


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -2785,4 +2785,11 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
         "upper" -> size.toString,
         "invalidValue" -> pos.toString))
   }
+
+  def notExistPartitionError(

Review Comment:
   Okay



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376068633


##########
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)

Review Comment:
   From the execution logic of `v1 command`, only one partition is supported
   https://github.com/apache/spark/blob/e499ba80be6828dcf0c34d3f06f1cd12321ce5b8/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala#L916-L920



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374574757


##########
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(

Review Comment:
   There is already a `createUnresolvedTable` function in this file, can we reuse it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1378701726


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ 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 - " +

Review Comment:
   Done.
   I have updated some logic of UT. Previously, I used the `exclude method` to exclude fields with `non-deterministic `values, but now I have changed it to `replace` with `a fixed template value` for better comparison of results. At present, the difference displayed by UT should be relatively clear. eg:
   
   `Created Time: Wed Nov 01 04:51:19 PDT 2023` => `Created Time: <created time>`
   `Last Access: UNKNOWN` => `Last Access: <last access>`
   `Created By: Spark` => `Created By: <created by>`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376087096


##########
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(

Review Comment:
   It seems difficult to reuse because
   - In existing functions `createUnresolvedTable `, `ctx: IdentifierReferenceContext,` represents `namespace. tablename`, it is a complete name.
   https://github.com/apache/spark/blob/e499ba80be6828dcf0c34d3f06f1cd12321ce5b8/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala#L2828-L2833
   
   - But in our scenario, `nsCtx: IdentifierReferenceContext` represents namespace, and `patternCtx: StringLitContext represents` tablename.
   <img width="809" alt="image" src="https://github.com/apache/spark/assets/15246973/c957c28f-b9ba-4714-971c-ced3e069bd13">
   If we wang to reuse it, we must use `nsCtx + patternCtx` to construct an `IdentifierReferenceContext`.



##########
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 {

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376969387


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2483,7 +2477,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       messageParameters = Map(
         "specKeys" -> specKeys,
         "partitionColumnNames" -> partitionColumnNames.mkString(", "),
-        "tableName" -> tableName))
+        "tableName" -> toSQLId(tableName)))

Review Comment:
   If this is not modified, the following issues will occur
   
   - When executes UT `org.apache.spark.sql.execution.command.v1.ShowTablesSuite.show table extended in multi partition key - the command's partition parameters are incomplete`, its error message is:
      ```
      org.apache.spark.sql.AnalysisException: Partition spec is invalid. The spec (id1) must match the partition spec (id1, id2) defined in table '`spark_catalog`.`ns1`.`tbl`'.
      ```
   https://github.com/apache/spark/blob/efa891c4ea869a7fac58eabf187ac8aeff1fcd38/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala#L1343-L1346
   
   - When executes `org.apache.spark.sql.execution.command.v2.ShowTablesSuite.show table extended in multi partition key - the command's partition parameters are incomplete`, its error message is:
      ```
      org.apache.spark.sql.AnalysisException: Partition spec is invalid. The spec (id1) must match the partition spec (id1, id2) defined in table 'test_catalog.ns1.tbl'.
      ```
   https://github.com/apache/spark/blob/efa891c4ea869a7fac58eabf187ac8aeff1fcd38/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala#L66
   
   
   
   The reason is that there are two entrances to calling the method `PartitioningUtils.requireExactMatchedPartitionSpec`, 
   <img width="466" alt="image" src="https://github.com/apache/spark/assets/15246973/41e3ecaa-d699-4b43-92da-53568dae3b10">
   
   and the format of the parameter `tableName: String` inside is not consistent, so it has been unified here. In this way, we can put this UT into `show table extended in multi partition key - the command's partition parameters are incomplete` to maintain consistency.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1382829435


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4055,19 +4055,34 @@ 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)
+    @inline def createUnresolvedTable(

Review Comment:
   Okay,I misunderstood the meaning, haha



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -4055,19 +4055,34 @@ 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)
+    @inline def createUnresolvedTable(

Review Comment:
   Okay,I misunderstood the meaning, haha



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] panbingkun commented on a diff in pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by GitBox <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r956708182


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -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))
   }
 

Review Comment:
   In this scenario, UnresolvedPartitionspec cannot be resolved to ResolvedPartitionspec, because the table is non-deterministic at this time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] MaxGekk commented on pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1428249555

   > are you able to take another look at this PR? 
   
   No, I haven't looked at this yet because of my wedding. :-)
   
   > this is an annoying feature gap that we need to work around currently for DSv2 sources :(
   
   @xkrogen If you really need this feature, please, review this PR. I will join to you slightly later.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1365551061


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -53,4 +82,95 @@ 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
+    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.isReadable) {
+      if (table.schema().nonEmpty) results.put("Schema", table.schema().treeString)

Review Comment:
   I think every table has schema, do we really need `if (table.isReadable)`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1364047573


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala:
##########
@@ -404,6 +404,13 @@ 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,

Review Comment:
   nit:
   ```suggestion
           ResolvedNamespace(catalog, ns),
           pattern,
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      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 '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 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: 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)
+    }
+  }
+
+  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")

Review Comment:
   ```suggestion
           assert(result.collect()(1)(1) === table1)
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      assert(result2.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result2.collect()(0).length == 4)
+      assert(result2.collect()(0)(3) ==
+        s"""Namespace: ns1.ns2

Review Comment:
   ```suggestion
           s"""Namespace: $namespace
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: ns1.ns2

Review Comment:
   ```suggestion
           s"""Namespace: $namespace
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      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 '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 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: 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)
+    }
+  }
+
+  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

Review Comment:
   ```suggestion
                |Table: $table1
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      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 '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 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: 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)
+    }
+  }
+
+  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

Review Comment:
   ```suggestion
                |Table: $table2
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      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 '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

Review Comment:
   ```suggestion
           s"""Namespace: $namespace
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      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 '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 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: ns1.ns2

Review Comment:
   ```suggestion
           s"""Namespace: $namespace
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      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 '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

Review Comment:
   ```suggestion
              |Table: $table
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      assert(result3.collect()(0)(3) ==
+        s"""Namespace: ns1.ns2
+           |Table: tbl

Review Comment:
   ```suggestion
              |Table: $table
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      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 '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 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: 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)
+    }
+  }
+
+  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")

Review Comment:
   ```suggestion
           assert(result.collect()(2)(1) === table2)
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: ns1.ns2
+           |Table: tbl

Review Comment:
   ```suggestion
              |Table: $table
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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 LIKE 'tb*'")
+      assert(result3.schema.fieldNames ===
+        Seq("namespace", "tableName", "isTemporary", "information"))
+      assert(result3.collect()(0).length == 4)
+      assert(result3.collect()(0)(3) ==
+        s"""Namespace: ns1.ns2

Review Comment:
   ```suggestion
           s"""Namespace: $namespace
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowTablesSuite.scala:
##########
@@ -49,57 +50,286 @@ 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 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`"))
+  }
 
-    val namespace = s"$catalog.ns1.ns2"
+  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"
-    withTable(s"$namespace.$table") {
-      sql(s"CREATE TABLE $namespace.$table (id bigint, data string) " +
-        s"$defaultUsing PARTITIONED BY (id)")
-
-      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)
+    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")
+      )
     }
   }
 
-  // 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")
+  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 FROM $catalog LIKE '*$table*'").collect()
+          sql(s"SHOW TABLE EXTENDED IN $catalog.$namespace LIKE '$table' PARTITION(id1 = 1)")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1200",
-        parameters = Map("name" -> "SHOW TABLE EXTENDED")
+        errorClass = "_LEGACY_ERROR_TEMP_1232",
+        parameters = Map(
+          "specKeys" -> "id1",
+          "partitionColumnNames" -> "id1, id2",
+          "tableName" -> s"$catalog.$namespace.$table")
       )
     }
   }
 
-  test("show table in a not existing namespace") {
-    val e = intercept[NoSuchNamespaceException] {
-      runShowTablesSql(s"SHOW TABLES IN $catalog.unknown", Seq())
+  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: 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 '$table'")
+      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

Review Comment:
   ```suggestion
              |Table: $table
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] closed pull request #37588: [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2
URL: https://github.com/apache/spark/pull/37588


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1765498209

   > @panbingkun Could you rebase it on the recent master, please.
   
   Okay, I will do it today.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on PR #37588:
URL: https://github.com/apache/spark/pull/37588#issuecomment-1763905260

   @panbingkun @MaxGekk Any plans to get this in at some point?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1372183519


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -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

Review Comment:
   The comment is wrong, the case belongs to Hive:
   ```json
     "_LEGACY_ERROR_TEMP_1251" : {
       "message" : [
         "<action> is not allowed on <tableName> since its partition metadata is not stored in the Hive metastore. To import this information into the metastore, run `msck repair table <tableName>`."
       ]
     },
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -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)

Review Comment:
   Don't think this good check. Let's follow existing convention, and create separate tests in V2, V1 and Hive traits.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -50,6 +50,16 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] {
           }
         case _ => command
       }
+    case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace),

Review Comment:
   Could you clarify why `ShowTableExtended ` doesn't extend `V2PartitionCommand`. And apparently why do we need this special case.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -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(

Review Comment:
   Not clear what does this checks, and how it belongs to the test. Could you put it to a separate test with proper title.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374453999


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolvePartitionSpec.scala:
##########
@@ -50,6 +50,16 @@ object ResolvePartitionSpec extends Rule[LogicalPlan] {
           }
         case _ => command
       }
+    case command @ ShowTableExtended(ResolvedNamespace(catalog: TableCatalog, namespace),

Review Comment:
   Based on the above suggestions, the modifications have been completed and we will add 2 commands: `ShowTablesExtended ` and `ShowTablePartition `
   Simultaneously delete: `ShowTableExtended`
   And in  `ResolveSessionCatalog`, conversion was made for the two new command mentioned above.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala:
##########
@@ -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(

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375241783


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala:
##########
@@ -2133,12 +2133,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
         "inputTypesLen" -> bound.inputTypes().length.toString))
   }
 
-  def commandUnsupportedInV2TableError(name: String): Throwable = {
-    new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1200",
-      messageParameters = Map("name" -> name))
-  }

Review Comment:
   To reduce the complexity of this PR, it will be placed in a separate PR to complete the renaming or removing of the error class-related operations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1374596994


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala:
##########
@@ -33,17 +41,36 @@ 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[ResolvedPartitionSpec] = 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))
+    if (partitionSpec.isEmpty) {
+      // Show the information of tables.
+      val identifiers = catalog.listTables(namespace.toArray)

Review Comment:
   can we add a TODO here? We need a new `listTable` overload that takes a pattern string.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375242555


##########
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(

Review Comment:
   @cloud-fan 
   Do we need this related modification?
   Because renaming the name to `ShowTablesExtended` or `ShowTablePartition` does not seem to convey the meaning of the following two commands at the same time.
   https://github.com/apache/spark/blob/57e73dafeb9ffd288e0ed4ad76b9549e7da33ad7/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4#L176-L177



##########
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)

Review Comment:
   Okay



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1375244071


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala:
##########
@@ -404,6 +404,17 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case ShowTables(ResolvedNamespace(catalog, ns), pattern, output) =>
       ShowTablesExec(output, catalog.asTableCatalog, ns, pattern) :: Nil
 
+    case ShowTablesExtended(
+        ResolvedNamespace(catalog, ns),
+        pattern,
+        output) =>
+      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()),

Review Comment:
   Okay, I'll separate it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1376094564


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowTablesSuite.scala:
##########
@@ -165,4 +154,222 @@ class ShowTablesSuite extends ShowTablesSuiteBase with CommandSuiteBase {
       }
     }
   }
+
+  test("show table extended in non-partitioned table") {

Review Comment:
   - v1 error class:  `_LEGACY_ERROR_TEMP_1251`
   https://github.com/apache/spark/blob/e499ba80be6828dcf0c34d3f06f1cd12321ce5b8/common/utils/src/main/resources/error/error-classes.json#L4849-L4853
   
   - v2 error class: `_LEGACY_ERROR_TEMP_1231`
   https://github.com/apache/spark/blob/e499ba80be6828dcf0c34d3f06f1cd12321ce5b8/common/utils/src/main/resources/error/error-classes.json#L4804-L4808
   
   Maybe we can merge it after this PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380241677


##########
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 = withOrigin(patternCtx) {
+      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)))

Review Comment:
   BTW we can inline `createUnresolvedTable` if it's only called once



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380240485


##########
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 = withOrigin(patternCtx) {
+      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)))

Review Comment:
   We should call `withIdentClause` to wrap `ShowTablePartition` as well



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1380256729


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    val views = sessionCatalog.listTempViews(db.get, pattern)
+    views.map { viewIdent =>
+      val database = viewIdent.database.getOrElse("")
+      val tableName = viewIdent.table
+      val isTemp = sessionCatalog.isTempView(viewIdent)

Review Comment:
   isn't it always true?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1381660916


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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) {

Review Comment:
   Yes, it was my bad.   I have corrected it.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    val views = sessionCatalog.listTempViews(db.get, pattern)
+    views.map { viewIdent =>
+      val database = viewIdent.database.getOrElse("")
+      val tableName = viewIdent.table
+      val isTemp = sessionCatalog.isTempView(viewIdent)

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1382833311


##########
dev/appveyor-install-dependencies.ps1:
##########
@@ -27,11 +27,13 @@ Function InstallR {
 
   $urlPath = ""
   $latestVer = $(ConvertFrom-JSON $(Invoke-WebRequest https://rversions.r-pkg.org/r-release-win).Content).version
+  $latestVer = "4.3.2"

Review Comment:
   Yeah, this URL `https://rversions.r-pkg.org/r-release-win` has returned the correct value, and I will remove it now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-33393][SQL] Support SHOW TABLE EXTENDED in v2 [spark]

Posted by "panbingkun (via GitHub)" <gi...@apache.org>.
panbingkun commented on code in PR #37588:
URL: https://github.com/apache/spark/pull/37588#discussion_r1390406012


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExtendedExec.scala:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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, 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.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 (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,
+          s"$information\n")
+        }
+      }
+
+    // fetch temp views, includes: global temp view, local temp view
+    val sessionCatalog = session.sessionState.catalog
+    val db = namespace match {
+      case Seq(db) => Some(db)
+      case _ => None
+    }
+    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
+  }
+
+  private def getTableDetails(
+      catalogName: String,
+      identifier: Identifier,
+      table: Table): String = {
+    val results = new mutable.LinkedHashMap[String, String]()
+
+    results.put("Catalog", catalogName)
+    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(

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org