Skip to content

[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

Closed
wants to merge 8 commits into from
Closed

Conversation

dongjoon-hyun
Copy link
Member

@dongjoon-hyun dongjoon-hyun commented Jul 9, 2016

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> 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.

@SparkQA
Copy link

SparkQA commented Jul 9, 2016

Test build #62020 has finished for PR 14114 at commit 46196a4.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@@ -442,6 +442,10 @@ class SessionCatalog(
name.database.isEmpty && tempTables.contains(formatTableName(name.table))
}

def listTemporaryTableOutput(name: String): Seq[Attribute] = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

rather than doing this, can we make getTableMetadata work for temp tables.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for review, @rxin .
I'll try again in getTableMetadata.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah. I remember that why I do this in this way. Basically, there are two barriers to reach getTableMetadata. Before making change, let me describe here.

  1. Redirecting: listColumns(table) -> listColumns(currentDatabase, tableName)
  2. Table existence failure: requireTableExists(dbName, tableName) in listColumns(currentDatabase, tableName).

Anyway, I'm trying to change the above barriers.

@dongjoon-hyun
Copy link
Member Author

Now, getTableMetadata is improved.
Thank you for that advice, @rxin .

@SparkQA
Copy link

SparkQA commented Jul 9, 2016

Test build #62035 has finished for PR 14114 at commit 9134a47.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 10, 2016

Test build #62036 has finished for PR 14114 at commit be0e69a.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 10, 2016

Test build #62039 has finished for PR 14114 at commit bb1204d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member Author

Hi, @rxin .
Now, it's ready for review again.

@dongjoon-hyun
Copy link
Member Author

Hi, @rxin .
Could you review this when you have some time?

val table = formatTableName(name.table)
if (name.database.isDefined || !tempTables.contains(table)) {
externalCatalog.tableExists(db, table)
if (name.database.getOrElse("").length == 0 && tempTables.contains(table)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this seems a litlte bit hacky - can we just write it as

if (name.database.isEmpty && tempTables.contains(table)) {
  // This is a temporary table
  true
} else {
  ...
}

@dongjoon-hyun
Copy link
Member Author

Thank you for comments! Both comment are tightly related to each other.
I see what is your point. Yep, implicit use of "" string is not a good idea. I'll fix it again according to the advices.

@SparkQA
Copy link

SparkQA commented Jul 11, 2016

Test build #62067 has finished for PR 14114 at commit af6692f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member Author

Now, it's back for review again.

@@ -425,10 +443,11 @@ class SessionCatalog(
def tableExists(name: TableIdentifier): Boolean = synchronized {
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure.

@dongjoon-hyun
Copy link
Member Author

Now, SessionCatalogSuite has tableExists and getTableMetadata testcases on temporary views.

@SparkQA
Copy link

SparkQA commented Jul 11, 2016

Test build #62077 has finished for PR 14114 at commit cac342e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 11, 2016

Test build #62079 has finished for PR 14114 at commit ac5f5cb.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member Author

Hi, @rxin.
Could you review this again?

@@ -138,7 +138,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
*/
@throws[AnalysisException]("table does not exist")
override def listColumns(tableName: String): Dataset[Column] = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This changes the contract of the listColumns(...) function. It now returns either a temporary view or a table in the current database. We have to document this! What happens when we have temporary table with the same name as a table in the current database?

Copy link
Contributor

Choose a reason for hiding this comment

The 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)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you again, @hvanhovell

  1. Yep. That is the purpose of this PR, to make the contract consistent with other APIs.
  2. The existence checking here is redundant because it call other listColumns. The callee will check that.

Copy link
Contributor

Choose a reason for hiding this comment

The 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...

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the proper wording, Spark temporary views?

scala> spark.range(10).createOrReplaceTempView("t1")

scala> spark.catalog.listTables().show
+----+--------+-----------+---------+-----------+
|name|database|description|tableType|isTemporary|
+----+--------+-----------+---------+-----------+
|  t1|    null|       null|TEMPORARY|       true|
+----+--------+-----------+---------+-----------+

Copy link
Member Author

@dongjoon-hyun dongjoon-hyun Jul 11, 2016

Choose a reason for hiding this comment

The 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: This includes all temporary tables.. It's ambiguous, but it is the current docs.

/**
 * Returns a list of tables in the current database.
 * This includes all temporary tables.
 *
 * @since 2.0.0
 */
def listTables(): Dataset[Table]

name = c.name,
dataType = c.dataType.catalogString,
nullable = c.nullable,
comment = Option(c.name)
Copy link
Contributor

Choose a reason for hiding this comment

The 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:
if (c.metadata.contains("comment")) Some(c.metadata.getString("comment")) else None

So I am fine with leaving this as it is...

Copy link
Member

Choose a reason for hiding this comment

The 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?

@dongjoon-hyun
Copy link
Member Author

@hvanhovell .
I rebased and updated one place for isTemporaryTable.
Thank you for in-depth review.

@hvanhovell
Copy link
Contributor

LGTM - pending jenkins

@SparkQA
Copy link

SparkQA commented Jul 11, 2016

Test build #62111 has finished for PR 14114 at commit e267713.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@hvanhovell
Copy link
Contributor

Merging to master/2.0. Thanks!

asfgit pushed a commit that referenced this pull request Jul 11, 2016
…mporary 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.

(cherry picked from commit 840853e)
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
@asfgit asfgit closed this in 840853e Jul 11, 2016
@SparkQA
Copy link

SparkQA commented Jul 11, 2016

Test build #62113 has finished for PR 14114 at commit 650ead2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member Author

Thank you for merging, @hvanhovell and @rxin .

@SparkQA
Copy link

SparkQA commented Jul 11, 2016

Test build #62114 has finished for PR 14114 at commit d1fa9ec.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants