@@ -52,18 +52,13 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
5252 /** Connection to hive metastore. Usages should lock on `this`. */
5353 protected [hive] val client = Hive .get(hive.hiveconf)
5454
55- // TODO: Use this everywhere instead of tuples or databaseName, tableName,.
56- /** A fully qualified identifier for a table (i.e., database.tableName) */
57- case class QualifiedTableName (database : String , name : String ) {
58- def toLowerCase = QualifiedTableName (database.toLowerCase, name.toLowerCase)
59- }
60-
6155 /** A cache of Spark SQL data source tables that have been accessed. */
62- protected [hive] val cachedDataSourceTables : LoadingCache [QualifiedTableName , LogicalPlan ] = {
63- val cacheLoader = new CacheLoader [QualifiedTableName , LogicalPlan ]() {
64- override def load (in : QualifiedTableName ): LogicalPlan = {
56+ protected [hive] val cachedDataSourceTables : LoadingCache [Seq [ String ] , LogicalPlan ] = {
57+ val cacheLoader = new CacheLoader [Seq [ String ] , LogicalPlan ]() {
58+ override def load (in : Seq [ String ] ): LogicalPlan = {
6559 logDebug(s " Creating new cached data source for $in" )
66- val table = client.getTable(in.database, in.name)
60+
61+ val table = client.getTable(in(0 ), in(1 ))
6762 val schemaString = table.getProperty(" spark.sql.sources.schema" )
6863 val userSpecifiedSchema =
6964 if (schemaString == null ) {
@@ -89,12 +84,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
8984 CacheBuilder .newBuilder().maximumSize(1000 ).build(cacheLoader)
9085 }
9186
92- def refreshTable (databaseName : String , tableName : String ): Unit = {
93- cachedDataSourceTables.refresh(QualifiedTableName (databaseName, tableName) .toLowerCase)
87+ def refreshTable (tableIdentifier : Seq [ String ] ): Unit = {
88+ cachedDataSourceTables.refresh(tableIdentifier.map(_ .toLowerCase()) )
9489 }
9590
96- def invalidateTable (databaseName : String , tableName : String ): Unit = {
97- cachedDataSourceTables.invalidate(QualifiedTableName (databaseName, tableName) .toLowerCase)
91+ def invalidateTable (tableIdentifier : Seq [ String ] ): Unit = {
92+ cachedDataSourceTables.invalidate(tableIdentifier.map(_ .toLowerCase()) )
9893 }
9994
10095 val caseSensitive : Boolean = false
@@ -146,7 +141,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
146141 val table = client.getTable(databaseName, tblName)
147142
148143 if (table.getProperty(" spark.sql.sources.provider" ) != null ) {
149- cachedDataSourceTables(QualifiedTableName (databaseName, tblName).toLowerCase )
144+ cachedDataSourceTables(Seq (databaseName, tblName))
150145 } else if (table.isView) {
151146 // if the unresolved relation is from hive view
152147 // parse the text into logic node.
0 commit comments