-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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
[SPARK-20194] Add support for partition pruning to in-memory catalog #17510
Changes from 2 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 |
---|---|---|
|
@@ -35,7 +35,7 @@ import org.apache.spark.sql.AnalysisException | |
import org.apache.spark.sql.catalyst.TableIdentifier | ||
import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException | ||
import org.apache.spark.sql.catalyst.catalog._ | ||
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName | ||
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.plans.logical.ColumnStat | ||
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap | ||
|
@@ -1039,37 +1039,14 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat | |
defaultTimeZoneId: String): Seq[CatalogTablePartition] = withClient { | ||
val rawTable = getRawTable(db, table) | ||
val catalogTable = restoreTableMetadata(rawTable) | ||
val partitionColumnNames = catalogTable.partitionColumnNames.toSet | ||
val nonPartitionPruningPredicates = predicates.filterNot { | ||
_.references.map(_.name).toSet.subsetOf(partitionColumnNames) | ||
} | ||
|
||
if (nonPartitionPruningPredicates.nonEmpty) { | ||
sys.error("Expected only partition pruning predicates: " + | ||
predicates.reduceLeft(And)) | ||
} | ||
val partColNameMap = buildLowerCasePartColNameMap(catalogTable) | ||
|
||
val partitionSchema = catalogTable.partitionSchema | ||
val partColNameMap = buildLowerCasePartColNameMap(getTable(db, table)) | ||
|
||
if (predicates.nonEmpty) { | ||
val clientPrunedPartitions = client.getPartitionsByFilter(rawTable, predicates).map { part => | ||
val clientPrunedPartitions = | ||
client.getPartitionsByFilter(rawTable, predicates).map { part => | ||
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. if 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. A similar optimization is done in the function itself: client.getPartitionsByFilter(), while Hive Shim_v0_12 delegates to I've now made the only piece of non-trivial code along that path lazy, so I think we're good. |
||
part.copy(spec = restorePartitionSpec(part.spec, partColNameMap)) | ||
} | ||
val boundPredicate = | ||
InterpretedPredicate.create(predicates.reduce(And).transform { | ||
case att: AttributeReference => | ||
val index = partitionSchema.indexWhere(_.name == att.name) | ||
BoundReference(index, partitionSchema(index).dataType, nullable = true) | ||
}) | ||
clientPrunedPartitions.filter { p => | ||
boundPredicate(p.toRow(partitionSchema, defaultTimeZoneId)) | ||
} | ||
} else { | ||
client.getPartitions(catalogTable).map { part => | ||
part.copy(spec = restorePartitionSpec(part.spec, partColNameMap)) | ||
} | ||
} | ||
prunePartitionsByFilter(catalogTable, clientPrunedPartitions, predicates, defaultTimeZoneId) | ||
} | ||
|
||
// -------------------------------------------------------------------------- | ||
|
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.
nit: code style