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/catalyst/src/main/scala | |
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/catalyst/src/main/scala')
-rw-r--r-- | sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 32 |
1 files changed, 25 insertions, 7 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index d88b5ffc05..c0ebb2b1fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -22,7 +22,7 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException @@ -253,9 +253,27 @@ class SessionCatalog( def getTableMetadata(name: TableIdentifier): CatalogTable = { val db = formatDatabaseName(name.database.getOrElse(getCurrentDatabase)) val table = formatTableName(name.table) - requireDbExists(db) - requireTableExists(TableIdentifier(table, Some(db))) - externalCatalog.getTable(db, table) + val tid = TableIdentifier(table) + if (isTemporaryTable(name)) { + CatalogTable( + identifier = tid, + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = tempTables(table).output.map { c => + CatalogColumn( + name = c.name, + dataType = c.dataType.catalogString, + nullable = c.nullable, + comment = Option(c.name) + ) + }, + properties = Map(), + viewText = None) + } else { + requireDbExists(db) + requireTableExists(TableIdentifier(table, Some(db))) + externalCatalog.getTable(db, table) + } } /** @@ -432,10 +450,10 @@ class SessionCatalog( def tableExists(name: TableIdentifier): Boolean = synchronized { val db = formatDatabaseName(name.database.getOrElse(currentDb)) val table = formatTableName(name.table) - if (name.database.isDefined || !tempTables.contains(table)) { - externalCatalog.tableExists(db, table) + if (isTemporaryTable(name)) { + true } else { - true // it's a temporary table + externalCatalog.tableExists(db, table) } } |