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



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala
##########
@@ -17,38 +17,91 @@
 
 package org.apache.spark.sql.execution.datasources.v2
 
+import com.google.common.base.Objects
+
+import org.apache.spark.SparkException
 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.catalyst.plans.physical.UnknownPartitioning
+import org.apache.spark.sql.catalyst.util.truncatedString
+import org.apache.spark.sql.connector.read.{InputPartition, 
PartitionReaderFactory, Scan, SupportsRuntimeFiltering}
+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,
+    runtimeFilters: Seq[Expression]) extends DataSourceV2ScanExecBase {
 
   @transient lazy val batch = scan.toBatch
 
   // 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 _ => false
+    case other: BatchScanExec =>
+      this.batch == other.batch && this.runtimeFilters == other.runtimeFilters
+    case _ =>
+      false
   }
 
-  override def hashCode(): Int = batch.hashCode()
+  override def hashCode(): Int = Objects.hashCode(batch, runtimeFilters)
 
   @transient override lazy val partitions: Seq[InputPartition] = 
batch.planInputPartitions()
 
+  @transient private lazy val filteredPartitions: Seq[InputPartition] = {
+    val dataSourceFilters = runtimeFilters.flatMap {
+      case DynamicPruningExpression(e) => 
DataSourceStrategy.translateRuntimeFilter(e)
+      case _ => None
+    }
+
+    if (dataSourceFilters.nonEmpty) {
+      // the cast is safe as runtime filters are only assigned if the scan can 
be filtered
+      val filterableScan = scan.asInstanceOf[SupportsRuntimeFiltering]
+      val canChangeNumPartitions = outputPartitioning match {
+        case UnknownPartitioning(0) => true
+        case _ => false
+      }
+      filterableScan.filter(dataSourceFilters.toArray, canChangeNumPartitions)
+
+      // call toBatch again to get new partitions
+      val newPartitions = scan.toBatch.planInputPartitions()

Review comment:
       We should update the comment in `Batch.java` since we are now calling 
`planInputPartitions` more than once and people might put some logic there that 
they assume is only run once.

##########
File path: 
sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/SupportsRuntimeFiltering.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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
+ * filter initially planned {@link InputPartition}s using predicates Spark 
infers at runtime.
+ * <p>
+ * Note that Spark will push runtime filters only if they are beneficial.

Review comment:
       How do we determine if it is beneficial?




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