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



##########
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

Review comment:
       Created SPARK-35900 to think about in the future.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to