cloud-fan commented on a change in pull request #32921:
URL: https://github.com/apache/spark/pull/32921#discussion_r654112138



##########
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:
       We can probably have a `SupportsPushDownV2Filters` to not break existing 
code.

##########
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:
       @JkSelf do you think it's possible to run 
`PlanAdaptiveDynamicPruningFilters` in `queryStagePreparationRules`?

##########
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:
       If we target this PR for 3.3, then I'm fine to use v1 Filter here and 
replace it with v2 Filter later, as there is plenty of time. Otherwise, I'd 
like to have v2 Filter first, to avoid releasing this API with v1 Filter and 
breaking it later.

##########
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:
       The problem is we can't add shuffles in the query optimization phase, 
and the options I can think of is:
   1. doing DPP in the stage preparation phase.
   2. improve the framework to allow adding shuffle in the query optimization 
phase.
   3. do not apply DPP if more shuffles are introduced.
   
   I think option 3 should be the simplest, but not sure if it's good enough 
for your use cases, @aokolnychyi 




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