-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-16458][SQL] SessionCatalog should support listColumns
for temporary tables
#14114
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
e3c59a0
631ae96
613bd15
ed5301d
ab43d47
e267713
650ead2
d1fa9ec
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The metadata can contain the comment, but it is a bit of a PITA to get out: So I am fine with leaving this as it is... There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is the reason we need to put the column name in the comment? |
||
) | ||
}, | ||
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 { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you update SessionCatalogSuite to reflect this behavior? I think we weren't checking temp tables in the past. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure. |
||
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) | ||
} | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -138,7 +138,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { | |
*/ | ||
@throws[AnalysisException]("table does not exist") | ||
override def listColumns(tableName: String): Dataset[Column] = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This changes the contract of the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should't we check if the table exists? (like the other listColumns(...) function) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you again, @hvanhovell
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The documentation in org.apache.spark.sql.catalog.Catalog: /**
* Returns a list of columns for the given table in the current database.
*
* @since 2.0.0
*/
@throws[AnalysisException]("table does not exist")
def listColumns(tableName: String): Dataset[Column] Should be updated... There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is the proper wording,
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I made mistakes too much. Sorry for that. I will change like the following: /**
* Returns a list of tables in the current database.
* This includes all temporary tables.
*
* @since 2.0.0
*/
def listTables(): Dataset[Table] |
||
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 => | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
same thing here - update SessionCatalogSuite.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yep.