diff options
author | Dongjoon Hyun <dongjoon@apache.org> | 2016-07-11 22:45:22 +0200 |
---|---|---|
committer | Herman van Hovell <hvanhovell@databricks.com> | 2016-07-11 22:45:22 +0200 |
commit | 840853ed06d63694bf98b21a889a960aac6ac0ac (patch) | |
tree | 579b5a42ca2daae6e2c0fbf4d33fd0216813d282 /sql/core | |
parent | ffcb6e055a28f36208ed058a42df09c154555332 (diff) | |
download | spark-840853ed06d63694bf98b21a889a960aac6ac0ac.tar.gz spark-840853ed06d63694bf98b21a889a960aac6ac0ac.tar.bz2 spark-840853ed06d63694bf98b21a889a960aac6ac0ac.zip |
[SPARK-16458][SQL] SessionCatalog should support `listColumns` for temporary tables
## What changes were proposed in this pull request?
Temporary tables are used frequently, but `spark.catalog.listColumns` does not support those tables. This PR make `SessionCatalog` supports temporary table column listing.
**Before**
```scala
scala> spark.range(10).createOrReplaceTempView("t1")
scala> spark.catalog.listTables().collect()
res1: Array[org.apache.spark.sql.catalog.Table] = Array(Table[name=`t1`, tableType=`TEMPORARY`, isTemporary=`true`])
scala> spark.catalog.listColumns("t1").collect()
org.apache.spark.sql.AnalysisException: Table `t1` does not exist in database `default`.;
```
**After**
```
scala> spark.catalog.listColumns("t1").collect()
res2: Array[org.apache.spark.sql.catalog.Column] = Array(Column[name='id', description='id', dataType='bigint', nullable='false', isPartition='false', isBucket='false'])
```
## How was this patch tested?
Pass the Jenkins tests including a new testcase.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes #14114 from dongjoon-hyun/SPARK-16458.
Diffstat (limited to 'sql/core')
3 files changed, 13 insertions, 3 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index 91ed9b3258..1aed245fdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -85,7 +85,8 @@ abstract class Catalog { def listFunctions(dbName: String): Dataset[Function] /** - * Returns a list of columns for the given table in the current database. + * Returns a list of columns for the given table in the current database or + * the given temporary table. * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 44babcc93a..a6ae6fe2aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -138,7 +138,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { */ @throws[AnalysisException]("table does not exist") override def listColumns(tableName: String): Dataset[Column] = { - listColumns(currentDatabase, tableName) + listColumns(TableIdentifier(tableName, None)) } /** @@ -147,7 +147,11 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { @throws[AnalysisException]("database or table does not exist") override def listColumns(dbName: String, tableName: String): Dataset[Column] = { requireTableExists(dbName, tableName) - val tableMetadata = sessionCatalog.getTableMetadata(TableIdentifier(tableName, Some(dbName))) + listColumns(TableIdentifier(tableName, Some(dbName))) + } + + private def listColumns(tableIdentifier: TableIdentifier): Dataset[Column] = { + val tableMetadata = sessionCatalog.getTableMetadata(tableIdentifier) val partitionColumnNames = tableMetadata.partitionColumnNames.toSet val bucketColumnNames = tableMetadata.bucketColumnNames.toSet val columns = tableMetadata.schema.map { c => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index d862e4cfa9..d75df56dd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -234,6 +234,11 @@ class CatalogSuite testListColumns("tab1", dbName = None) } + test("list columns in temporary table") { + createTempTable("temp1") + spark.catalog.listColumns("temp1") + } + test("list columns in database") { createDatabase("db1") createTable("tab1", Some("db1")) |