-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18028][SQL] simplify TableFileCatalog #15568
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,36 +20,30 @@ package org.apache.spark.sql.execution.datasources | |
| 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 [[FileCatalog]] for a metastore catalog table. | ||
| * | ||
| * @param sparkSession a [[SparkSession]] | ||
| * @param db the table's database name | ||
| * @param table the table's (unqualified) name | ||
| * @param partitionSchema the schema of a partitioned table's partition columns | ||
| * @param table the metadata of the table | ||
| * @param sizeInBytes the table's data size in bytes | ||
| * @param fileStatusCache optional cache implementation to use for file listing | ||
| */ | ||
| class TableFileCatalog( | ||
| sparkSession: SparkSession, | ||
| db: String, | ||
| table: String, | ||
| partitionSchema: Option[StructType], | ||
| val table: CatalogTable, | ||
| override val sizeInBytes: Long) extends FileCatalog { | ||
|
|
||
| protected val hadoopConf = sparkSession.sessionState.newHadoopConf | ||
|
|
||
| private val fileStatusCache = FileStatusCache.newCache(sparkSession) | ||
|
|
||
| private val externalCatalog = sparkSession.sharedState.externalCatalog | ||
| assert(table.identifier.database.isDefined, | ||
| "The table identifier must be qualified in TableFileCatalog") | ||
|
|
||
| private val catalogTable = externalCatalog.getTable(db, table) | ||
|
|
||
| private val baseLocation = catalogTable.storage.locationUri | ||
| private val baseLocation = table.storage.locationUri | ||
|
|
||
| override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq | ||
|
|
||
|
|
@@ -66,24 +60,32 @@ class TableFileCatalog( | |
| * @param filters partition-pruning filters | ||
| */ | ||
| def filterPartitions(filters: Seq[Expression]): ListingFileCatalog = { | ||
| val parameters = baseLocation | ||
| .map(loc => Map(PartitioningAwareFileCatalog.BASE_PATH_PARAM -> loc)) | ||
| .getOrElse(Map.empty) | ||
| partitionSchema match { | ||
| case Some(schema) => | ||
| val selectedPartitions = externalCatalog.listPartitionsByFilter(db, table, filters) | ||
| val partitions = selectedPartitions.map { p => | ||
| PartitionPath(p.toRow(schema), p.storage.locationUri.get) | ||
| } | ||
| val partitionSpec = PartitionSpec(schema, partitions) | ||
| new PrunedTableFileCatalog( | ||
| sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec) | ||
| case None => | ||
| new ListingFileCatalog(sparkSession, rootPaths, parameters, None, fileStatusCache) | ||
| if (table.partitionColumnNames.nonEmpty) { | ||
| val selectedPartitions = sparkSession.sessionState.catalog.listPartitionsByFilter( | ||
| table.identifier, filters) | ||
| val partitionSchema = table.partitionSchema | ||
| val partitions = selectedPartitions.map { p => | ||
| PartitionPath(p.toRow(partitionSchema), p.storage.locationUri.get) | ||
| } | ||
| val partitionSpec = PartitionSpec(partitionSchema, partitions) | ||
| new PrunedTableFileCatalog( | ||
| sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec) | ||
| } else { | ||
| new ListingFileCatalog(sparkSession, rootPaths, table.storage.properties, None) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @cloud-fan, here I have a question why should we remove the
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. seems it was a mistake. Can you send a PR to add it? thanks!
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. OK, sure. Thanks for your answer. |
||
| } | ||
| } | ||
|
|
||
| override def inputFiles: Array[String] = filterPartitions(Nil).inputFiles | ||
|
|
||
| // `TableFileCatalog` 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: TableFileCatalog => this.table.identifier == other.table.identifier | ||
| case _ => false | ||
| } | ||
|
|
||
| override def hashCode(): Int = table.identifier.hashCode() | ||
| } | ||
|
|
||
| /** | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you for adding this, @cloud-fan ! I think I can use this in #15302 .