@@ -106,8 +106,14 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
106106 }
107107
108108 override def refreshTable (databaseName : String , tableName : String ): Unit = {
109- // refresh table does not eagerly reload the cache. It just invalidate the cache.
109+ // refreshTable does not eagerly reload the cache. It just invalidate the cache.
110110 // Next time when we use the table, it will be populated in the cache.
111+ // Since we also cache ParquetRealtions converted from Hive Parquet tables and
112+ // adding converted ParquetRealtions into the cache is not defined in the load function
113+ // of the cache (instead, we add the cache entry in convertToParquetRelation),
114+ // it is better at here to invalidate the cache to avoid confusing waring logs from the
115+ // cache loader (e.g. cannot find data source provider, which is only defined for
116+ // data source table.).
111117 invalidateTable(databaseName, tableName)
112118 }
113119
@@ -226,21 +232,27 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
226232 QualifiedTableName (metastoreRelation.databaseName, metastoreRelation.tableName)
227233
228234 def getCached (
229- tableIdentifier : QualifiedTableName ,
230- pathsInMetastore : Seq [String ],
231- schemaInMetastore : StructType ,
232- partitionSpecInMetastore : Option [PartitionSpec ]): Option [LogicalRelation ] = {
235+ tableIdentifier : QualifiedTableName ,
236+ pathsInMetastore : Seq [String ],
237+ schemaInMetastore : StructType ,
238+ partitionSpecInMetastore : Option [PartitionSpec ]): Option [LogicalRelation ] = {
233239 cachedDataSourceTables.getIfPresent(tableIdentifier) match {
234240 case null => None // Cache miss
235- case logical @ LogicalRelation (parquetRelation : ParquetRelation2 ) =>
241+ case logical@ LogicalRelation (parquetRelation : ParquetRelation2 ) =>
236242 // If we have the same paths, same schema, and same partition spec,
237243 // we will use the cached Parquet Relation.
238244 val useCached =
239- parquetRelation.paths == pathsInMetastore &&
245+ parquetRelation.paths.toSet == pathsInMetastore.toSet &&
240246 logical.schema.sameType(metastoreSchema) &&
241247 parquetRelation.maybePartitionSpec == partitionSpecInMetastore
242248
243- if (useCached) Some (logical) else None
249+ if (useCached) {
250+ Some (logical)
251+ } else {
252+ // If the cached relation is not updated, we invalidate it right away.
253+ cachedDataSourceTables.invalidate(tableIdentifier)
254+ None
255+ }
244256 case other =>
245257 logWarning(
246258 s " ${metastoreRelation.databaseName}. ${metastoreRelation.tableName} shold be stored " +
0 commit comments