diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index e544a7c8767e..a159590ebed1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -289,7 +289,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat if (partitionSpec.nonEmpty) { throw QueryCompilationErrors.describeDoesNotSupportPartitionForV2TablesError() } - DescribeTableExec(output, r.table, isExtended) :: Nil + DescribeTableExec(output, r, isExtended) :: Nil case DescribeColumn(_: ResolvedTable, column, isExtended, output) => column match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index c20189efc91f..d75b7037c388 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -21,13 +21,17 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.ResolvedTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, Table} case class DescribeTableExec( output: Seq[Attribute], - table: Table, + resolvedTable: ResolvedTable, isExtended: Boolean) extends LeafV2CommandExec { + + private val table: Table = resolvedTable.table + override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() addSchema(rows) @@ -43,7 +47,9 @@ case class DescribeTableExec( private def addTableDetails(rows: ArrayBuffer[InternalRow]): Unit = { rows += emptyRow() rows += toCatalystRow("# Detailed Table Information", "", "") - rows += toCatalystRow("Name", table.name(), "") + rows += toCatalystRow("Catalog", resolvedTable.catalog.name(), "") + rows += toCatalystRow("Database", resolvedTable.identifier.namespace().mkString("."), "") + rows += toCatalystRow("Table", resolvedTable.identifier.name(), "") CatalogV2Util.TABLE_RESERVED_PROPERTIES.foreach(propKey => { if (table.properties.containsKey(propKey)) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index b64ed080d8bf..68ec97b0cb5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -120,13 +120,13 @@ class DataSourceV2SQLSuite } test("DescribeTable extended using v2 catalog") { - spark.sql("CREATE TABLE testcat.table_name (id bigint, data string)" + + spark.sql("CREATE TABLE testcat.default.table_name (id bigint, data string)" + " USING foo" + " PARTITIONED BY (id)" + " TBLPROPERTIES ('bar'='baz')" + " COMMENT 'this is a test table'" + " LOCATION 'file:/tmp/testcat/table_name'") - val descriptionDf = spark.sql("DESCRIBE TABLE EXTENDED testcat.table_name") + val descriptionDf = spark.sql("DESCRIBE TABLE EXTENDED testcat.default.table_name") assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( ("col_name", StringType), @@ -146,7 +146,9 @@ class DataSourceV2SQLSuite Array("_partition", "string", "Partition key used to store the row"), Array("", "", ""), Array("# Detailed Table Information", "", ""), - Array("Name", "testcat.table_name", ""), + Array("Catalog", "testcat", ""), + Array("Database", "default", ""), + Array("Table", "table_name", ""), Array("Comment", "this is a test table", ""), Array("Location", "file:/tmp/testcat/table_name", ""), Array("Provider", "foo", ""),