Skip to content

Commit f9f98dd

Browse files
authored
Custom CatalogFileIndex (apache-spark-on-k8s#364)
1 parent 5ccc040 commit f9f98dd

File tree

8 files changed

+185
-95
lines changed

8 files changed

+185
-95
lines changed

.circleci/config.yml

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -201,11 +201,11 @@ jobs:
201201
<<: *defaults
202202
steps:
203203
# Saves us from recompiling every time...
204-
- restore_cache:
205-
keys:
206-
- v1-build-sbt-{{ .Branch }}-{{ .Revision }}
207-
- v1-build-sbt-{{ .Branch }}-
208-
- v1-build-sbt-master-
204+
#- restore_cache:
205+
#keys:
206+
#- v1-build-sbt-{{ .Branch }}-{{ .Revision }}
207+
#- v1-build-sbt-{{ .Branch }}-
208+
#- v1-build-sbt-master-
209209
- *checkout-code
210210
- run:
211211
name: Hard link cache contents into current build directory

sql/catalyst/src/main/scala/org/apache/spark/sql/internal/StaticSQLConf.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,12 @@ object StaticSQLConf {
3333
.stringConf
3434
.createWithDefault(Utils.resolveURI("spark-warehouse").toString)
3535

36+
val CATALOG_FILE_INDEX_IMPLEMENTATION =
37+
buildStaticConf("spark.sql.catalogFileIndexImplementation")
38+
.internal()
39+
.stringConf
40+
.createWithDefault("hive")
41+
3642
val CATALOG_IMPLEMENTATION = buildStaticConf("spark.sql.catalogImplementation")
3743
.internal()
3844
.stringConf

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala

Lines changed: 36 additions & 83 deletions
Original file line numberDiff line numberDiff line change
@@ -17,107 +17,60 @@
1717

1818
package org.apache.spark.sql.execution.datasources
1919

20-
import java.net.URI
21-
22-
import org.apache.hadoop.conf.Configuration
23-
import org.apache.hadoop.fs.Path
20+
import scala.util.control.NonFatal
2421

22+
import org.apache.spark.SparkConf
2523
import org.apache.spark.sql.SparkSession
2624
import org.apache.spark.sql.catalyst.catalog.CatalogTable
2725
import org.apache.spark.sql.catalyst.expressions._
28-
import org.apache.spark.sql.types.StructType
29-
26+
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_FILE_INDEX_IMPLEMENTATION
27+
import org.apache.spark.util.Utils
3028

3129
/**
3230
* A [[FileIndex]] for a metastore catalog table.
33-
*
34-
* @param sparkSession a [[SparkSession]]
35-
* @param table the metadata of the table
36-
* @param sizeInBytes the table's data size in bytes
3731
*/
38-
class CatalogFileIndex(
39-
sparkSession: SparkSession,
40-
val table: CatalogTable,
41-
override val sizeInBytes: Long) extends FileIndex {
42-
43-
protected val hadoopConf: Configuration = sparkSession.sessionState.newHadoopConf()
32+
trait CatalogFileIndex extends FileIndex {
4433

45-
/** Globally shared (not exclusive to this table) cache for file statuses to speed up listing. */
46-
private val fileStatusCache = FileStatusCache.getOrCreate(sparkSession)
47-
48-
assert(table.identifier.database.isDefined,
49-
"The table identifier must be qualified in CatalogFileIndex")
34+
/**
35+
* Returns a [[FileIndex]] for this table restricted to the subset of partitions
36+
* specified by the given partition-pruning filters.
37+
*
38+
* @param filters partition-pruning filters
39+
*/
40+
def filterPartitions(filters: Seq[Expression]): FileIndex
5041

51-
private val baseLocation: Option[URI] = table.storage.locationUri
42+
}
5243

53-
override def partitionSchema: StructType = table.partitionSchema
44+
trait CatalogFileIndexFactory {
5445

55-
override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq
46+
/**
47+
* Creates [[CatalogFileIndex]] for given table
48+
*/
49+
def create(
50+
spark: SparkSession,
51+
catalogTable: CatalogTable,
52+
tableSize: Long): CatalogFileIndex
5653

57-
override def listFiles(
58-
partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
59-
filterPartitions(partitionFilters).listFiles(Nil, dataFilters)
60-
}
54+
}
6155

62-
override def refresh(): Unit = fileStatusCache.invalidateAll()
56+
object CatalogFileIndexFactory {
6357

64-
/**
65-
* Returns a [[InMemoryFileIndex]] for this table restricted to the subset of partitions
66-
* specified by the given partition-pruning filters.
67-
*
68-
* @param filters partition-pruning filters
69-
*/
70-
def filterPartitions(filters: Seq[Expression]): InMemoryFileIndex = {
71-
if (table.partitionColumnNames.nonEmpty) {
72-
val startTime = System.nanoTime()
73-
val selectedPartitions = sparkSession.sessionState.catalog.listPartitionsByFilter(
74-
table.identifier, filters)
75-
val partitions = selectedPartitions.map { p =>
76-
val path = new Path(p.location)
77-
val fs = path.getFileSystem(hadoopConf)
78-
PartitionPath(
79-
p.toRow(partitionSchema, sparkSession.sessionState.conf.sessionLocalTimeZone),
80-
path.makeQualified(fs.getUri, fs.getWorkingDirectory))
81-
}
82-
val partitionSpec = PartitionSpec(partitionSchema, partitions)
83-
val timeNs = System.nanoTime() - startTime
84-
new PrunedInMemoryFileIndex(
85-
sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec, Option(timeNs))
86-
} else {
87-
new InMemoryFileIndex(
88-
sparkSession, rootPaths, table.storage.properties, userSpecifiedSchema = None)
58+
def reflect[T <: CatalogFileIndexFactory](conf: SparkConf): T = {
59+
val className = fileIndexClassName(conf)
60+
try {
61+
val ctor = Utils.classForName(className).getDeclaredConstructor()
62+
ctor.newInstance().asInstanceOf[T]
63+
} catch {
64+
case NonFatal(e) =>
65+
throw new IllegalArgumentException(s"Error while instantiating '$className':", e)
8966
}
9067
}
9168

92-
override def inputFiles: Array[String] = filterPartitions(Nil).inputFiles
93-
94-
// `CatalogFileIndex` may be a member of `HadoopFsRelation`, `HadoopFsRelation` may be a member
95-
// of `LogicalRelation`, and `LogicalRelation` may be used as the cache key. So we need to
96-
// implement `equals` and `hashCode` here, to make it work with cache lookup.
97-
override def equals(o: Any): Boolean = o match {
98-
case other: CatalogFileIndex => this.table.identifier == other.table.identifier
99-
case _ => false
69+
private def fileIndexClassName(conf: SparkConf): String = {
70+
conf.get(CATALOG_FILE_INDEX_IMPLEMENTATION) match {
71+
case "hive" => "org.apache.spark.sql.execution.datasources.HiveCatalogFileIndexFactory"
72+
case name => name
73+
}
10074
}
10175

102-
override def hashCode(): Int = table.identifier.hashCode()
10376
}
104-
105-
/**
106-
* An override of the standard HDFS listing based catalog, that overrides the partition spec with
107-
* the information from the metastore.
108-
*
109-
* @param tableBasePath The default base path of the Hive metastore table
110-
* @param partitionSpec The partition specifications from Hive metastore
111-
*/
112-
private class PrunedInMemoryFileIndex(
113-
sparkSession: SparkSession,
114-
tableBasePath: Path,
115-
fileStatusCache: FileStatusCache,
116-
override val partitionSpec: PartitionSpec,
117-
override val metadataOpsTimeNs: Option[Long])
118-
extends InMemoryFileIndex(
119-
sparkSession,
120-
partitionSpec.partitions.map(_.path),
121-
Map.empty,
122-
Some(partitionSpec.partitionColumns),
123-
fileStatusCache)

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -89,6 +89,8 @@ case class DataSource(
8989

9090
case class SourceInfo(name: String, schema: StructType, partitionColumns: Seq[String])
9191

92+
lazy val fileIndexFactory: CatalogFileIndexFactory =
93+
CatalogFileIndexFactory.reflect(sparkSession.sparkContext.conf)
9294
lazy val providingClass: Class[_] =
9395
DataSource.lookupDataSource(className, sparkSession.sessionState.conf)
9496
lazy val sourceInfo: SourceInfo = sourceSchema()
@@ -361,7 +363,7 @@ case class DataSource(
361363
catalogTable.get.partitionColumnNames.nonEmpty
362364
val (fileCatalog, dataSchema, partitionSchema) = if (useCatalogFileIndex) {
363365
val defaultTableSize = sparkSession.sessionState.conf.defaultSizeInBytes
364-
val index = new CatalogFileIndex(
366+
val index = fileIndexFactory.create(
365367
sparkSession,
366368
catalogTable.get,
367369
catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(defaultTableSize))
Lines changed: 129 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,129 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution.datasources
19+
20+
import java.net.URI
21+
22+
import org.apache.hadoop.conf.Configuration
23+
import org.apache.hadoop.fs.Path
24+
25+
import org.apache.spark.sql.SparkSession
26+
import org.apache.spark.sql.catalyst.catalog.CatalogTable
27+
import org.apache.spark.sql.catalyst.expressions._
28+
import org.apache.spark.sql.types.StructType
29+
30+
31+
/**
32+
* A [[FileIndex]] for a metastore catalog table.
33+
*
34+
* @param sparkSession a [[SparkSession]]
35+
* @param table the metadata of the table
36+
* @param sizeInBytes the table's data size in bytes
37+
*/
38+
class HiveCatalogFileIndex(
39+
sparkSession: SparkSession,
40+
val table: CatalogTable,
41+
override val sizeInBytes: Long) extends CatalogFileIndex {
42+
43+
protected val hadoopConf: Configuration = sparkSession.sessionState.newHadoopConf()
44+
45+
/** Globally shared (not exclusive to this table) cache for file statuses to speed up listing. */
46+
private val fileStatusCache = FileStatusCache.getOrCreate(sparkSession)
47+
48+
assert(table.identifier.database.isDefined,
49+
"The table identifier must be qualified in CatalogFileIndex")
50+
51+
private val baseLocation: Option[URI] = table.storage.locationUri
52+
53+
override def partitionSchema: StructType = table.partitionSchema
54+
55+
override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq
56+
57+
override def listFiles(
58+
partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
59+
filterPartitions(partitionFilters).listFiles(Nil, dataFilters)
60+
}
61+
62+
override def refresh(): Unit = fileStatusCache.invalidateAll()
63+
64+
/**
65+
* Returns a [[InMemoryFileIndex]] for this table restricted to the subset of partitions
66+
* specified by the given partition-pruning filters.
67+
*
68+
* @param filters partition-pruning filters
69+
*/
70+
def filterPartitions(filters: Seq[Expression]): InMemoryFileIndex = {
71+
if (table.partitionColumnNames.nonEmpty) {
72+
val startTime = System.nanoTime()
73+
val selectedPartitions = sparkSession.sessionState.catalog.listPartitionsByFilter(
74+
table.identifier, filters)
75+
val partitions = selectedPartitions.map { p =>
76+
val path = new Path(p.location)
77+
val fs = path.getFileSystem(hadoopConf)
78+
PartitionPath(
79+
p.toRow(partitionSchema, sparkSession.sessionState.conf.sessionLocalTimeZone),
80+
path.makeQualified(fs.getUri, fs.getWorkingDirectory))
81+
}
82+
val partitionSpec = PartitionSpec(partitionSchema, partitions)
83+
val timeNs = System.nanoTime() - startTime
84+
new PrunedInMemoryFileIndex(
85+
sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec, Option(timeNs))
86+
} else {
87+
new InMemoryFileIndex(
88+
sparkSession, rootPaths, table.storage.properties, userSpecifiedSchema = None)
89+
}
90+
}
91+
92+
override def inputFiles: Array[String] = filterPartitions(Nil).inputFiles
93+
94+
// `CatalogFileIndex` may be a member of `HadoopFsRelation`, `HadoopFsRelation` may be a member
95+
// of `LogicalRelation`, and `LogicalRelation` may be used as the cache key. So we need to
96+
// implement `equals` and `hashCode` here, to make it work with cache lookup.
97+
override def equals(o: Any): Boolean = o match {
98+
case other: HiveCatalogFileIndex => this.table.identifier == other.table.identifier
99+
case _ => false
100+
}
101+
102+
override def hashCode(): Int = table.identifier.hashCode()
103+
}
104+
105+
class HiveCatalogFileIndexFactory extends CatalogFileIndexFactory {
106+
override def create(
107+
spark: SparkSession, catalogTable: CatalogTable, tableSize: Long): CatalogFileIndex =
108+
new HiveCatalogFileIndex(spark, catalogTable, tableSize)
109+
}
110+
111+
/**
112+
* An override of the standard HDFS listing based catalog, that overrides the partition spec with
113+
* the information from the metastore.
114+
*
115+
* @param tableBasePath The default base path of the Hive metastore table
116+
* @param partitionSpec The partition specifications from Hive metastore
117+
*/
118+
private class PrunedInMemoryFileIndex(
119+
sparkSession: SparkSession,
120+
tableBasePath: Path,
121+
fileStatusCache: FileStatusCache,
122+
override val partitionSpec: PartitionSpec,
123+
override val metadataOpsTimeNs: Option[Long])
124+
extends InMemoryFileIndex(
125+
sparkSession,
126+
partitionSpec.partitions.map(_.path),
127+
Map.empty,
128+
Some(partitionSpec.partitionColumns),
129+
fileStatusCache)

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -156,7 +156,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
156156
val logicalRelation = cached.getOrElse {
157157
val sizeInBytes = relation.stats.sizeInBytes.toLong
158158
val fileIndex = {
159-
val index = new CatalogFileIndex(sparkSession, relation.tableMeta, sizeInBytes)
159+
val index = new HiveCatalogFileIndex(sparkSession, relation.tableMeta, sizeInBytes)
160160
if (lazyPruningEnabled) {
161161
index
162162
} else {

sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import org.apache.spark.sql.{AnalysisException, Dataset, QueryTest, SaveMode}
2323
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
2424
import org.apache.spark.sql.catalyst.parser.ParseException
2525
import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec
26-
import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation}
26+
import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, HiveCatalogFileIndex, LogicalRelation}
2727
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
2828
import org.apache.spark.sql.hive.test.TestHiveSingleton
2929
import org.apache.spark.sql.test.SQLTestUtils
@@ -320,7 +320,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
320320
withTable("test") {
321321
sql("CREATE TABLE test(i int) PARTITIONED BY (p int) STORED AS parquet")
322322
val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test")
323-
val catalogFileIndex = new CatalogFileIndex(spark, tableMeta, 0)
323+
val catalogFileIndex = new HiveCatalogFileIndex(spark, tableMeta, 0)
324324

325325
val dataSchema = StructType(tableMeta.schema.filterNot { f =>
326326
tableMeta.partitionColumnNames.contains(f.name)
@@ -338,7 +338,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
338338

339339
assert(spark.sharedState.cacheManager.lookupCachedData(plan).isDefined)
340340

341-
val sameCatalog = new CatalogFileIndex(spark, tableMeta, 0)
341+
val sameCatalog = new HiveCatalogFileIndex(spark, tableMeta, 0)
342342
val sameRelation = HadoopFsRelation(
343343
location = sameCatalog,
344344
partitionSchema = tableMeta.partitionSchema,

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._
2323
import org.apache.spark.sql.catalyst.dsl.plans._
2424
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
2525
import org.apache.spark.sql.catalyst.rules.RuleExecutor
26-
import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions}
26+
import org.apache.spark.sql.execution.datasources._
2727
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
2828
import org.apache.spark.sql.hive.test.TestHiveSingleton
2929
import org.apache.spark.sql.test.SQLTestUtils
@@ -46,7 +46,7 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te
4646
|LOCATION '${dir.toURI}'""".stripMargin)
4747

4848
val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test")
49-
val catalogFileIndex = new CatalogFileIndex(spark, tableMeta, 0)
49+
val catalogFileIndex = new HiveCatalogFileIndex(spark, tableMeta, 0)
5050

5151
val dataSchema = StructType(tableMeta.schema.filterNot { f =>
5252
tableMeta.partitionColumnNames.contains(f.name)

0 commit comments

Comments
 (0)