Repository: spark
Updated Branches:
  refs/heads/master 3ad99f166 -> 90d3b91f4


http://git-wip-us.apache.org/repos/asf/spark/blob/90d3b91f/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index d1de863..624ab74 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -200,7 +200,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
       val rootPaths: Seq[Path] = if (lazyPruningEnabled) {
         Seq(metastoreRelation.hiveQlTable.getDataLocation)
       } else {
-        // By convention (for example, see TableFileCatalog), the definition 
of a
+        // By convention (for example, see CatalogFileIndex), the definition 
of a
         // partitioned table's paths depends on whether that table has any 
actual partitions.
         // Partitioned tables without partitions use the location of the 
table's base path.
         // Partitioned tables with partitions use the locations of those 
partitions' data
@@ -227,7 +227,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: 
SparkSession) extends Log
       val logicalRelation = cached.getOrElse {
         val sizeInBytes = metastoreRelation.statistics.sizeInBytes.toLong
         val fileCatalog = {
-          val catalog = new TableFileCatalog(
+          val catalog = new CatalogFileIndex(
             sparkSession, metastoreRelation.catalogTable, sizeInBytes)
           if (lazyPruningEnabled) {
             catalog

http://git-wip-us.apache.org/repos/asf/spark/blob/90d3b91f/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
index ecdf4f1..fc35304 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.{AnalysisException, Dataset, 
QueryTest, SaveMode}
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
-import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, 
LogicalRelation, TableFileCatalog}
+import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, 
HadoopFsRelation, LogicalRelation}
 import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
 import org.apache.spark.sql.hive.test.TestHiveSingleton
 import org.apache.spark.sql.test.SQLTestUtils
@@ -321,17 +321,17 @@ class CachedTableSuite extends QueryTest with 
SQLTestUtils with TestHiveSingleto
     sql("DROP TABLE cachedTable")
   }
 
-  test("cache a table using TableFileCatalog") {
+  test("cache a table using CatalogFileIndex") {
     withTable("test") {
       sql("CREATE TABLE test(i int) PARTITIONED BY (p int) STORED AS parquet")
       val tableMeta = spark.sharedState.externalCatalog.getTable("default", 
"test")
-      val tableFileCatalog = new TableFileCatalog(spark, tableMeta, 0)
+      val catalogFileIndex = new CatalogFileIndex(spark, tableMeta, 0)
 
       val dataSchema = StructType(tableMeta.schema.filterNot { f =>
         tableMeta.partitionColumnNames.contains(f.name)
       })
       val relation = HadoopFsRelation(
-        location = tableFileCatalog,
+        location = catalogFileIndex,
         partitionSchema = tableMeta.partitionSchema,
         dataSchema = dataSchema,
         bucketSpec = None,
@@ -343,7 +343,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils 
with TestHiveSingleto
 
       assert(spark.sharedState.cacheManager.lookupCachedData(plan).isDefined)
 
-      val sameCatalog = new TableFileCatalog(spark, tableMeta, 0)
+      val sameCatalog = new CatalogFileIndex(spark, tableMeta, 0)
       val sameRelation = HadoopFsRelation(
         location = sameCatalog,
         partitionSchema = tableMeta.partitionSchema,

http://git-wip-us.apache.org/repos/asf/spark/blob/90d3b91f/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
index 476383a..d8e31c4 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala
@@ -256,7 +256,7 @@ class PartitionedTablePerfStatsSuite
           // of doing plan cache validation based on the entire partition set.
           HiveCatalogMetrics.reset()
           assert(spark.sql("select * from test where partCol1 = 999").count() 
== 0)
-          // 5 from table resolution, another 5 from ListingFileCatalog
+          // 5 from table resolution, another 5 from InMemoryFileIndex
           assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 10)
           assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/90d3b91f/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala
index 59639aa..cdbc26c 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
 import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, 
Project}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
-import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, 
LogicalRelation, PruneFileSourcePartitions, TableFileCatalog}
+import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, 
HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions}
 import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
 import org.apache.spark.sql.hive.test.TestHiveSingleton
 import org.apache.spark.sql.test.SQLTestUtils
@@ -45,13 +45,13 @@ class PruneFileSourcePartitionsSuite extends QueryTest with 
SQLTestUtils with Te
             |LOCATION '${dir.getAbsolutePath}'""".stripMargin)
 
         val tableMeta = spark.sharedState.externalCatalog.getTable("default", 
"test")
-        val tableFileCatalog = new TableFileCatalog(spark, tableMeta, 0)
+        val catalogFileIndex = new CatalogFileIndex(spark, tableMeta, 0)
 
         val dataSchema = StructType(tableMeta.schema.filterNot { f =>
           tableMeta.partitionColumnNames.contains(f.name)
         })
         val relation = HadoopFsRelation(
-          location = tableFileCatalog,
+          location = catalogFileIndex,
           partitionSchema = tableMeta.partitionSchema,
           dataSchema = dataSchema,
           bucketSpec = None,


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to