Skip to content

Commit

Permalink
[SPARK-20439][SQL][BACKPORT-2.1] Fix Catalog API listTables and getTa…
Browse files Browse the repository at this point in the history
…ble when failed to fetch table metadata

### What changes were proposed in this pull request?

This PR is to backport #17730 to Spark 2.1
--- --
`spark.catalog.listTables` and `spark.catalog.getTable` does not work if we are unable to retrieve table metadata due to any reason (e.g., table serde class is not accessible or the table type is not accepted by Spark SQL). After this PR, the APIs still return the corresponding Table without the description and tableType)

### How was this patch tested?
Added a test case

Author: Xiao Li <[email protected]>

Closes #17760 from gatorsmile/backport-17730.
  • Loading branch information
gatorsmile authored and cloud-fan committed Apr 26, 2017
1 parent 8460b09 commit 6696ad0
Show file tree
Hide file tree
Showing 2 changed files with 31 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.internal

import scala.collection.JavaConverters._
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal

import org.apache.spark.annotation.Experimental
import org.apache.spark.sql._
Expand Down Expand Up @@ -99,14 +100,27 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
CatalogImpl.makeDataset(tables, sparkSession)
}

/**
* Returns a Table for the given table/view or temporary view.
*
* Note that this function requires the table already exists in the Catalog.
*
* If the table metadata retrieval failed due to any reason (e.g., table serde class
* is not accessible or the table type is not accepted by Spark SQL), this function
* still returns the corresponding Table without the description and tableType)
*/
private def makeTable(tableIdent: TableIdentifier): Table = {
val metadata = sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdent)
val metadata = try {
Some(sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdent))
} catch {
case NonFatal(_) => None
}
val isTemp = sessionCatalog.isTemporaryTable(tableIdent)
new Table(
name = tableIdent.table,
database = metadata.identifier.database.orNull,
description = metadata.comment.orNull,
tableType = if (isTemp) "TEMPORARY" else metadata.tableType.name,
database = metadata.map(_.identifier.database).getOrElse(tableIdent.database).orNull,
description = metadata.map(_.comment.orNull).orNull,
tableType = if (isTemp) "TEMPORARY" else metadata.map(_.tableType.name).orNull,
isTemporary = isTemp)
}

Expand Down Expand Up @@ -197,7 +211,11 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
* `AnalysisException` when no `Table` can be found.
*/
override def getTable(dbName: String, tableName: String): Table = {
makeTable(TableIdentifier(tableName, Option(dbName)))
if (tableExists(dbName, tableName)) {
makeTable(TableIdentifier(tableName, Option(dbName)))
} else {
throw new AnalysisException(s"Table or view '$tableName' not found in database '$dbName'")
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1060,6 +1060,14 @@ class HiveDDLSuite
s"CREATE INDEX $indexName ON TABLE $tabName (a) AS 'COMPACT' WITH DEFERRED REBUILD")
val indexTabName =
spark.sessionState.catalog.listTables("default", s"*$indexName*").head.table

// Even if index tables exist, listTables and getTable APIs should still work
checkAnswer(
spark.catalog.listTables().toDF(),
Row(indexTabName, "default", null, null, false) ::
Row(tabName, "default", null, "MANAGED", false) :: Nil)
assert(spark.catalog.getTable("default", indexTabName).name === indexTabName)

intercept[TableAlreadyExistsException] {
sql(s"CREATE TABLE $indexTabName(b int)")
}
Expand Down

0 comments on commit 6696ad0

Please sign in to comment.