forked from apache-spark-on-k8s/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Custom CatalogFileIndex (apache-spark-on-k8s#364)
- Loading branch information
1 parent
5ccc040
commit f9f98dd
Showing
8 changed files
with
185 additions
and
95 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
129 changes: 129 additions & 0 deletions
129
...core/src/main/scala/org/apache/spark/sql/execution/datasources/HiveCatalogFileIndex.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,129 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.sql.execution.datasources | ||
|
||
import java.net.URI | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.Path | ||
|
||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.catalyst.catalog.CatalogTable | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.types.StructType | ||
|
||
|
||
/** | ||
* A [[FileIndex]] for a metastore catalog table. | ||
* | ||
* @param sparkSession a [[SparkSession]] | ||
* @param table the metadata of the table | ||
* @param sizeInBytes the table's data size in bytes | ||
*/ | ||
class HiveCatalogFileIndex( | ||
sparkSession: SparkSession, | ||
val table: CatalogTable, | ||
override val sizeInBytes: Long) extends CatalogFileIndex { | ||
|
||
protected val hadoopConf: Configuration = sparkSession.sessionState.newHadoopConf() | ||
|
||
/** Globally shared (not exclusive to this table) cache for file statuses to speed up listing. */ | ||
private val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) | ||
|
||
assert(table.identifier.database.isDefined, | ||
"The table identifier must be qualified in CatalogFileIndex") | ||
|
||
private val baseLocation: Option[URI] = table.storage.locationUri | ||
|
||
override def partitionSchema: StructType = table.partitionSchema | ||
|
||
override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq | ||
|
||
override def listFiles( | ||
partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { | ||
filterPartitions(partitionFilters).listFiles(Nil, dataFilters) | ||
} | ||
|
||
override def refresh(): Unit = fileStatusCache.invalidateAll() | ||
|
||
/** | ||
* Returns a [[InMemoryFileIndex]] for this table restricted to the subset of partitions | ||
* specified by the given partition-pruning filters. | ||
* | ||
* @param filters partition-pruning filters | ||
*/ | ||
def filterPartitions(filters: Seq[Expression]): InMemoryFileIndex = { | ||
if (table.partitionColumnNames.nonEmpty) { | ||
val startTime = System.nanoTime() | ||
val selectedPartitions = sparkSession.sessionState.catalog.listPartitionsByFilter( | ||
table.identifier, filters) | ||
val partitions = selectedPartitions.map { p => | ||
val path = new Path(p.location) | ||
val fs = path.getFileSystem(hadoopConf) | ||
PartitionPath( | ||
p.toRow(partitionSchema, sparkSession.sessionState.conf.sessionLocalTimeZone), | ||
path.makeQualified(fs.getUri, fs.getWorkingDirectory)) | ||
} | ||
val partitionSpec = PartitionSpec(partitionSchema, partitions) | ||
val timeNs = System.nanoTime() - startTime | ||
new PrunedInMemoryFileIndex( | ||
sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec, Option(timeNs)) | ||
} else { | ||
new InMemoryFileIndex( | ||
sparkSession, rootPaths, table.storage.properties, userSpecifiedSchema = None) | ||
} | ||
} | ||
|
||
override def inputFiles: Array[String] = filterPartitions(Nil).inputFiles | ||
|
||
// `CatalogFileIndex` may be a member of `HadoopFsRelation`, `HadoopFsRelation` may be a member | ||
// of `LogicalRelation`, and `LogicalRelation` may be used as the cache key. So we need to | ||
// implement `equals` and `hashCode` here, to make it work with cache lookup. | ||
override def equals(o: Any): Boolean = o match { | ||
case other: HiveCatalogFileIndex => this.table.identifier == other.table.identifier | ||
case _ => false | ||
} | ||
|
||
override def hashCode(): Int = table.identifier.hashCode() | ||
} | ||
|
||
class HiveCatalogFileIndexFactory extends CatalogFileIndexFactory { | ||
override def create( | ||
spark: SparkSession, catalogTable: CatalogTable, tableSize: Long): CatalogFileIndex = | ||
new HiveCatalogFileIndex(spark, catalogTable, tableSize) | ||
} | ||
|
||
/** | ||
* An override of the standard HDFS listing based catalog, that overrides the partition spec with | ||
* the information from the metastore. | ||
* | ||
* @param tableBasePath The default base path of the Hive metastore table | ||
* @param partitionSpec The partition specifications from Hive metastore | ||
*/ | ||
private class PrunedInMemoryFileIndex( | ||
sparkSession: SparkSession, | ||
tableBasePath: Path, | ||
fileStatusCache: FileStatusCache, | ||
override val partitionSpec: PartitionSpec, | ||
override val metadataOpsTimeNs: Option[Long]) | ||
extends InMemoryFileIndex( | ||
sparkSession, | ||
partitionSpec.partitions.map(_.path), | ||
Map.empty, | ||
Some(partitionSpec.partitionColumns), | ||
fileStatusCache) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters