spark git commit: [SPARK-16458][SQL] SessionCatalog should support `listColumns` for temporary tables
Repository: spark Updated Branches: refs/heads/branch-2.0 72cf74324 -> aea33bf05 [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 HyunCloses #14114 from dongjoon-hyun/SPARK-16458. (cherry picked from commit 840853ed06d63694bf98b21a889a960aac6ac0ac) Signed-off-by: Herman van Hovell Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/aea33bf0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/aea33bf0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/aea33bf0 Branch: refs/heads/branch-2.0 Commit: aea33bf05fef49683eaa858f653aad5a30f37e4a Parents: 72cf743 Author: Dongjoon Hyun Authored: Mon Jul 11 22:45:22 2016 +0200 Committer: Herman van Hovell Committed: Mon Jul 11 22:45:49 2016 +0200 -- .../sql/catalyst/catalog/SessionCatalog.scala | 32 ++- .../catalyst/catalog/SessionCatalogSuite.scala | 33 .../org/apache/spark/sql/catalog/Catalog.scala | 3 +- .../apache/spark/sql/internal/CatalogImpl.scala | 8 +++-- .../spark/sql/internal/CatalogSuite.scala | 5 +++ 5 files changed, 71 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/aea33bf0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala -- 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 0f7e15f..3a2e574 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) } }
spark git commit: [SPARK-16458][SQL] SessionCatalog should support `listColumns` for temporary tables
Repository: spark Updated Branches: refs/heads/master ffcb6e055 -> 840853ed0 [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 HyunCloses #14114 from dongjoon-hyun/SPARK-16458. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/840853ed Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/840853ed Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/840853ed Branch: refs/heads/master Commit: 840853ed06d63694bf98b21a889a960aac6ac0ac Parents: ffcb6e0 Author: Dongjoon Hyun Authored: Mon Jul 11 22:45:22 2016 +0200 Committer: Herman van Hovell Committed: Mon Jul 11 22:45:22 2016 +0200 -- .../sql/catalyst/catalog/SessionCatalog.scala | 32 ++- .../catalyst/catalog/SessionCatalogSuite.scala | 33 .../org/apache/spark/sql/catalog/Catalog.scala | 3 +- .../apache/spark/sql/internal/CatalogImpl.scala | 8 +++-- .../spark/sql/internal/CatalogSuite.scala | 5 +++ 5 files changed, 71 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/840853ed/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala -- 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 d88b5ff..c0ebb2b 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) } } http://git-wip-us.apache.org/repos/asf/spark/blob/840853ed/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala -- diff --git