sunchao commented on a change in pull request #32921:
URL: https://github.com/apache/spark/pull/32921#discussion_r654092399



##########
File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsDynamicFiltering.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.connector.read;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.sources.Filter;
+
+/**
+ * A mix-in interface for {@link Scan}. Data sources can implement this 
interface if they can
+ * dynamically filter {@link InputPartition}s that were originally planned 
using predicates
+ * Spark infers by reusing parts of the query.
+ *
+ * @since 3.2.0
+ */
+@Experimental
+public interface SupportsDynamicFiltering extends Scan {
+  /**
+   * Returns attributes this scan can be dynamically filtered by.
+   * <p>
+   * Spark will call {@link #filter(Filter[])} if it can derive a dynamic
+   * predicate for any of the filter attributes.
+   */
+  NamedReference[] filterAttributes();
+
+  /**
+   * Dynamically filters this scan.
+   * <p>
+   * The provided expressions must be interpreted as a set of filters that are 
ANDed together.
+   * Implementations may use the filters to prune originally planned {@link 
InputPartition}s.
+   * <p>
+   * Spark will call {@link Scan#toBatch()}, {@link 
Scan#toMicroBatchStream(String)} or
+   * {@link Scan#toContinuousStream(String)} again after filtering the scan 
dynamically.
+   * The newly produced {@link Batch} or its streaming alternative may report 
a subset of
+   * originally planned {@link InputPartition}s.
+   *
+   * @param filters dynamic filters
+   */
+  void filter(Filter[] filters);

Review comment:
       @dbtsai used to have a PR on this: 
https://github.com/dbtsai/spark/pull/10 but hasn't been updated for a while now.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
##########
@@ -96,6 +96,7 @@ case class AdaptiveSparkPlanExec(
   @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq(
     PlanAdaptiveDynamicPruningFilters(this),
     ReuseAdaptiveSubquery(context.subqueryCache),
+    PrepareScans,

Review comment:
       also curious why `PlanAdaptiveDynamicPruningFilters` is in 
`queryStageOptimizerRules` but not `queryStagePreparationRules`. Any particular 
reason?
   
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -17,38 +17,84 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
+import com.google.common.base.Objects
+
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.connector.read.{InputPartition, 
PartitionReaderFactory, Scan}
+import org.apache.spark.sql.connector.read.{InputPartition, 
PartitionReaderFactory, Scan, SupportsDynamicFiltering}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 
 /**
  * Physical plan node for scanning a batch of data from a data source v2.
  */
 case class BatchScanExec(
     output: Seq[AttributeReference],
-    @transient scan: Scan) extends DataSourceV2ScanExecBase {
+    @transient scan: Scan,
+    dynamicFilters: Seq[Expression]) extends DataSourceV2ScanExecBase {
 
   @transient lazy val batch = scan.toBatch
 
+  private lazy val runnableDynamicFilters = dynamicFilters.collect {
+    case e: DynamicPruningExpression if e.child != Literal.TrueLiteral => e
+  }
+
   // TODO: unify the equal/hashCode implementation for all data source v2 
query plans.
   override def equals(other: Any): Boolean = other match {
-    case other: BatchScanExec => this.batch == other.batch
+    case other: BatchScanExec =>
+      this.batch == other.batch && this.dynamicFilters == other.dynamicFilters
     case _ => false
   }
 
-  override def hashCode(): Int = batch.hashCode()
+  override def hashCode(): Int = Objects.hashCode(batch, dynamicFilters)
+
+  @transient private lazy val originalPartitions = batch.planInputPartitions()
+  @transient private var filteredPartitions: Option[Seq[InputPartition]] = None
 
-  @transient override lazy val partitions: Seq[InputPartition] = 
batch.planInputPartitions()
+  override def partitions: Seq[InputPartition] = 
filteredPartitions.getOrElse(originalPartitions)
 
-  override lazy val readerFactory: PartitionReaderFactory = 
batch.createReaderFactory()
+  private lazy val originalReaderFactory = batch.createReaderFactory()
+  private var filteredReaderFactory: Option[PartitionReaderFactory] = None
+
+  override def readerFactory: PartitionReaderFactory = {
+    filteredReaderFactory.getOrElse(originalReaderFactory)
+  }
 
   override lazy val inputRDD: RDD[InternalRow] = {
     new DataSourceRDD(sparkContext, partitions, readerFactory, 
supportsColumnar, customMetrics)
   }
 
   override def doCanonicalize(): BatchScanExec = {
-    this.copy(output = output.map(QueryPlan.normalizeExpressions(_, output)))
+    this.copy(
+      output = output.map(QueryPlan.normalizeExpressions(_, output)),
+      dynamicFilters = QueryPlan.normalizePredicates(
+        dynamicFilters.filterNot(_ == 
DynamicPruningExpression(Literal.TrueLiteral)),
+        output))
+  }
+
+  override protected def doPrepare(): Unit = {
+    if (runnableDynamicFilters.nonEmpty) {
+      // TODO: this triggers a broadcast which we don't need
+      waitForSubqueries()
+
+      val dataSourceFilters = runnableDynamicFilters.flatMap {
+        case DynamicPruningExpression(e) => 
DataSourceStrategy.translateDynamicFilter(e)
+        case _ => None
+      }
+
+      if (dataSourceFilters.isEmpty) {
+        logWarning("Skipping dynamic filtering, could not derive source 
filters")
+        return
+      }
+
+      val filterableScan = scan.asInstanceOf[SupportsDynamicFiltering]
+      filterableScan.filter(dataSourceFilters.toArray)
+
+      val filteredBatch = scan.toBatch
+
+      filteredPartitions = Some(filteredBatch.planInputPartitions())

Review comment:
       I see. In that case maybe it's useful to make this a bit more clear for 
data source implementors (I'm not sure if there's enough signal for them that 
`planInputPartitions` will be called twice).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to