-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-36351][SQL] Refactor filter push down in file source v2 #33650
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 19 commits
3eec42e
c3e07ac
f95b4a0
feea946
e9d598f
a6ae1c5
ff8a9d4
8f06107
c541315
eaebb4c
f61caa0
e04428b
ab6187c
5b41c61
3b9e2c6
73eea33
095a7b4
68ace26
f3b4d22
4700c08
3085bdf
da9fe2f
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 |
|---|---|---|
| @@ -0,0 +1,33 @@ | ||
| /* | ||
| * 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.internal.connector | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.Expression | ||
| import org.apache.spark.sql.sources.Filter | ||
|
|
||
| /** | ||
| * A mix-in interface for {@link FileScanBuilder}. This can be used to push down filters to | ||
| * FileIndex in the format of catalyst Expression. | ||
| */ | ||
| trait SupportsPushDownCatalystFilters { | ||
| /** | ||
| * Pushes down filters to FileIndex in the format of catalyst Expression. The filters will be | ||
| * separated into partition filters and data filters. The data filters that are pushed to the | ||
| * data source and the date filters that need to be evaluated after scanning are returned. | ||
| */ | ||
| def pushCatalystFilters(partitionFilters: Seq[Expression]): (Array[Filter], Seq[Expression]) | ||
cloud-fan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,6 +28,7 @@ import org.json4s.jackson.Serialization | |
| import org.apache.spark.SparkUpgradeException | ||
| import org.apache.spark.sql.{SPARK_LEGACY_DATETIME, SPARK_LEGACY_INT96, SPARK_VERSION_METADATA_KEY} | ||
| import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils} | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, ExpressionSet, PredicateHelper} | ||
| import org.apache.spark.sql.catalyst.util.RebaseDateTime | ||
| import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} | ||
| import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions | ||
|
|
@@ -39,7 +40,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap | |
| import org.apache.spark.util.Utils | ||
|
|
||
|
|
||
| object DataSourceUtils { | ||
| object DataSourceUtils extends PredicateHelper { | ||
| /** | ||
| * The key to use for storing partitionBy columns as options. | ||
| */ | ||
|
|
@@ -242,4 +243,16 @@ object DataSourceUtils { | |
| options | ||
| } | ||
| } | ||
|
|
||
| def getPartitionFiltersAndDataFilters( | ||
| partitionColumns: Seq[Attribute], | ||
|
||
| normalizedFilters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { | ||
| val partitionSet = AttributeSet(partitionColumns) | ||
| val (partitionFilters, dataFilters) = normalizedFilters.partition(f => | ||
| f.references.subsetOf(partitionSet) | ||
| ) | ||
| val extraPartitionFilter = | ||
| dataFilters.flatMap(extractPredicatesWithinOutputSet(_, partitionSet)) | ||
| (ExpressionSet(partitionFilters ++ extraPartitionFilter).toSeq, dataFilters) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,19 +16,30 @@ | |
| */ | ||
| package org.apache.spark.sql.execution.datasources.v2 | ||
|
|
||
| import org.apache.spark.sql.SparkSession | ||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.sql.{sources, SparkSession} | ||
| import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} | ||
| import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownRequiredColumns} | ||
| import org.apache.spark.sql.execution.datasources.{PartitioningAwareFileIndex, PartitioningUtils} | ||
| import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, DataSourceUtils, PartitioningAwareFileIndex, PartitioningUtils} | ||
| import org.apache.spark.sql.internal.connector.SupportsPushDownCatalystFilters | ||
| import org.apache.spark.sql.sources.Filter | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
| abstract class FileScanBuilder( | ||
| sparkSession: SparkSession, | ||
| fileIndex: PartitioningAwareFileIndex, | ||
| dataSchema: StructType) extends ScanBuilder with SupportsPushDownRequiredColumns { | ||
| dataSchema: StructType) | ||
| extends ScanBuilder | ||
| with SupportsPushDownRequiredColumns | ||
| with SupportsPushDownCatalystFilters { | ||
| private val partitionSchema = fileIndex.partitionSchema | ||
| private val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis | ||
| protected val supportsNestedSchemaPruning = false | ||
| protected var requiredSchema = StructType(dataSchema.fields ++ partitionSchema.fields) | ||
| protected var partitionFilters = Seq.empty[Expression] | ||
| protected var dataFilters = Seq.empty[Expression] | ||
cloud-fan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| protected var pushedDataFilters = Array.empty[Filter] | ||
|
|
||
| override def pruneColumns(requiredSchema: StructType): Unit = { | ||
| // [SPARK-30107] While `requiredSchema` might have pruned nested columns, | ||
|
|
@@ -48,7 +59,7 @@ abstract class FileScanBuilder( | |
| StructType(fields) | ||
| } | ||
|
|
||
| protected def readPartitionSchema(): StructType = { | ||
| def readPartitionSchema(): StructType = { | ||
| val requiredNameSet = createRequiredNameSet() | ||
| val fields = partitionSchema.fields.filter { field => | ||
| val colName = PartitioningUtils.getColName(field, isCaseSensitive) | ||
|
|
@@ -57,6 +68,32 @@ abstract class FileScanBuilder( | |
| StructType(fields) | ||
| } | ||
|
|
||
| override def pushCatalystFilters(filters: Seq[Expression]): (Array[Filter], Seq[Expression]) = { | ||
| val partitionColNames = | ||
| partitionSchema.fields.map(PartitioningUtils.getColName(_, isCaseSensitive)).toSet | ||
|
||
| val partitionCol = filters.flatMap { expr => | ||
| expr.collect { | ||
| case attr: AttributeReference if partitionColNames.contains(attr.name) => | ||
| attr | ||
| } | ||
| } | ||
| val (partitionFilters, dataFilters) = | ||
| DataSourceUtils.getPartitionFiltersAndDataFilters(partitionCol, filters) | ||
| this.partitionFilters = partitionFilters | ||
| this.dataFilters = dataFilters | ||
| val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] | ||
| for (filterExpr <- dataFilters) { | ||
| val translated = DataSourceStrategy.translateFilter(filterExpr, true) | ||
| if (translated.nonEmpty) { | ||
| translatedFilters += translated.get | ||
| } | ||
| } | ||
| pushedDataFilters = pushDataFilters(translatedFilters.toArray) | ||
| (pushedDataFilters, dataFilters) | ||
| } | ||
|
|
||
| protected def pushDataFilters(dataFilters: Array[Filter]): Array[Filter] = Array.empty[Filter] | ||
huaxingao marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| private def createRequiredNameSet(): Set[String] = | ||
| requiredSchema.fields.map(PartitioningUtils.getColName(_, isCaseSensitive)).toSet | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,9 +25,7 @@ import org.apache.spark.sql.catalyst.util.CharVarcharUtils | |
| import org.apache.spark.sql.connector.expressions.FieldReference | ||
| import org.apache.spark.sql.connector.expressions.aggregate.Aggregation | ||
| import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownAggregates, SupportsPushDownFilters, SupportsPushDownRequiredColumns} | ||
| import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} | ||
| import org.apache.spark.sql.execution.datasources.DataSourceStrategy | ||
| import org.apache.spark.sql.execution.datasources.PushableColumnWithoutNestedColumn | ||
| import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, PushableColumnWithoutNestedColumn} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.sources | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
@@ -39,9 +37,9 @@ object PushDownUtils extends PredicateHelper { | |
| * @return pushed filter and post-scan filters. | ||
| */ | ||
| def pushFilters( | ||
| scanBuilder: ScanBuilder, | ||
| scanBuilderHolder: ScanBuilderHolder, | ||
cloud-fan marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| filters: Seq[Expression]): (Seq[sources.Filter], Seq[Expression]) = { | ||
| scanBuilder match { | ||
| scanBuilderHolder.builder match { | ||
| case r: SupportsPushDownFilters => | ||
| // A map from translated data source leaf node filters to original catalyst filter | ||
| // expressions. For a `And`/`Or` predicate, it is possible that the predicate is partially | ||
|
|
@@ -71,6 +69,9 @@ object PushDownUtils extends PredicateHelper { | |
| } | ||
| (r.pushedFilters(), (untranslatableExprs ++ postScanFilters).toSeq) | ||
|
|
||
| case f: FileScanBuilder => | ||
| val (pushedFilters, postScanFilters) = f.pushCatalystFilters(filters) | ||
| (pushedFilters, postScanFilters) | ||
| case _ => (Nil, filters) | ||
| } | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.