Skip to content

Commit

Permalink
fix.
Browse files Browse the repository at this point in the history
  • Loading branch information
gatorsmile committed Apr 25, 2017
1 parent 4279665 commit f910b0f
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 f910b0f

Please sign in to comment.