You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/11/25 06:53:54 UTC

[GitHub] [arrow-datafusion] jackwener opened a new pull request, #4365: reimplement `push_down_filter` to remove global-state

jackwener opened a new pull request, #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365

   # Which issue does this PR close?
   
   Close #4266
   Part of #4267
   
   # Rationale for this change
   
   <!--
    Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
    Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.  
   -->
   
   # What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   # Are these changes tested?
   
   <!--
   We typically require tests for all PRs in order to:
   1. Prevent the code from being accidentally broken by subsequent changes
   2. Serve as another way to document the expected behavior of the code
   
   If tests are not included in your PR, please explain why (for example, are they covered by existing tests)?
   -->
   
   # Are there any user-facing changes?
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!--
   If there are any breaking changes to public APIs, please add the `api change` label.
   -->


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034477311


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]
                     })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                    .collect::<HashMap<_, _>>();
+
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
+
+                // optimize inner
+                from_plan(
+                    filter.input(),
+                    &(**filter.input()).expressions(),
+                    &[new_filter],
+                )?
             }
-
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let replace_map = input
+                        .schema()
+                        .fields()
+                        .iter()
+                        .map(|field| (field.qualified_name(), col(field.name())))
+                        .collect::<HashMap<_, _>>();
+
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {
+                let mut new_scan_filters = scan.filters.clone();
+                let mut new_predicate = vec![];
+
+                let filter_predicates = utils::split_conjunction_owned(
+                    utils::cnf_rewrite(filter.predicate().clone()),
+                );
+
+                for filter_expr in &filter_predicates {
+                    let (preserve_filter_node, add_to_provider) =
+                        match scan.source.supports_filter_pushdown(filter_expr)? {
+                            TableProviderFilterPushDown::Unsupported => (true, false),
+                            TableProviderFilterPushDown::Inexact => (true, true),
+                            TableProviderFilterPushDown::Exact => (false, true),
+                        };
+                    if preserve_filter_node {
+                        new_predicate.push(filter_expr.clone());
+                    }
+                    if add_to_provider {
+                        // avoid reduplicated filter expr.
+                        if new_scan_filters.contains(filter_expr) {
+                            continue;
+                        }
+                        new_scan_filters.push(filter_expr.clone());
+                    }
+                }
 
-impl OptimizerRule for FilterPushDown {
-    fn name(&self) -> &str {
-        "filter_push_down"
-    }
+                let new_scan = LogicalPlan::TableScan(TableScan {
+                    source: scan.source.clone(),
+                    projection: scan.projection.clone(),
+                    projected_schema: scan.projected_schema.clone(),
+                    table_name: scan.table_name.clone(),
+                    filters: new_scan_filters,
+                    fetch: scan.fetch,
+                });
+
+                match conjunction(new_predicate) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_scan),
+                    )?),
+                    None => new_scan,
+                }
+            }
+            _ => plan.clone(),
+        };
 

Review Comment:
   I think the `LogicalPlan::Repartition `is also filter commutable. it is missing case.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb merged pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
alamb merged PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1330514170

   I plan to test this PR against the IOx test suite later today 


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034729820


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -1159,26 +1064,24 @@ mod tests {
             \n      Limit: skip=0, fetch=20\
             \n        Projection: test.a, test.b\
             \n          TableScan: test";
-        assert_optimized_plan_eq(&plan, expected);
-        Ok(())
+        assert_optimized_plan_eq(&plan, expected)
     }
 
     #[test]
     fn union_all() -> Result<()> {
         let table_scan = test_table_scan()?;
-        let plan = LogicalPlanBuilder::from(table_scan.clone())
-            .union(LogicalPlanBuilder::from(table_scan).build()?)?
+        let table_scan2 = test_table_scan_with_name("test2")?;

Review Comment:
   Could you please add one more UT to cover the case that the union's inputs have different named Field case ? There maybe implicit type conversions here.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034284068


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down

Review Comment:
   Yes, I think we can.
   Current code is just for refactor, I don't include extra enhancement.
   Related issue to track problems like this https://github.com/apache/arrow-datafusion/issues/4413



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034477311


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]
                     })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                    .collect::<HashMap<_, _>>();
+
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
+
+                // optimize inner
+                from_plan(
+                    filter.input(),
+                    &(**filter.input()).expressions(),
+                    &[new_filter],
+                )?
             }
-
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let replace_map = input
+                        .schema()
+                        .fields()
+                        .iter()
+                        .map(|field| (field.qualified_name(), col(field.name())))
+                        .collect::<HashMap<_, _>>();
+
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {
+                let mut new_scan_filters = scan.filters.clone();
+                let mut new_predicate = vec![];
+
+                let filter_predicates = utils::split_conjunction_owned(
+                    utils::cnf_rewrite(filter.predicate().clone()),
+                );
+
+                for filter_expr in &filter_predicates {
+                    let (preserve_filter_node, add_to_provider) =
+                        match scan.source.supports_filter_pushdown(filter_expr)? {
+                            TableProviderFilterPushDown::Unsupported => (true, false),
+                            TableProviderFilterPushDown::Inexact => (true, true),
+                            TableProviderFilterPushDown::Exact => (false, true),
+                        };
+                    if preserve_filter_node {
+                        new_predicate.push(filter_expr.clone());
+                    }
+                    if add_to_provider {
+                        // avoid reduplicated filter expr.
+                        if new_scan_filters.contains(filter_expr) {
+                            continue;
+                        }
+                        new_scan_filters.push(filter_expr.clone());
+                    }
+                }
 
-impl OptimizerRule for FilterPushDown {
-    fn name(&self) -> &str {
-        "filter_push_down"
-    }
+                let new_scan = LogicalPlan::TableScan(TableScan {
+                    source: scan.source.clone(),
+                    projection: scan.projection.clone(),
+                    projected_schema: scan.projected_schema.clone(),
+                    table_name: scan.table_name.clone(),
+                    filters: new_scan_filters,
+                    fetch: scan.fetch,
+                });
+
+                match conjunction(new_predicate) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_scan),
+                    )?),
+                    None => new_scan,
+                }
+            }
+            _ => plan.clone(),
+        };
 

Review Comment:
   I think the `LogicalPlan::Repartition` and `LogicalPlan::Distinct` are also filter commutable. They are the missing cases.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034632791


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]

Review Comment:
   has resolved this bug, and add UT, thanks @mingmwang !



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034741280


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -1159,26 +1064,24 @@ mod tests {
             \n      Limit: skip=0, fetch=20\
             \n        Projection: test.a, test.b\
             \n          TableScan: test";
-        assert_optimized_plan_eq(&plan, expected);
-        Ok(())
+        assert_optimized_plan_eq(&plan, expected)
     }
 
     #[test]
     fn union_all() -> Result<()> {
         let table_scan = test_table_scan()?;
-        let plan = LogicalPlanBuilder::from(table_scan.clone())
-            .union(LogicalPlanBuilder::from(table_scan).build()?)?
+        let table_scan2 = test_table_scan_with_name("test2")?;

Review Comment:
   add `test_union_different_schema`



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1330829748

   I plan to review the code and test failure in this PR later today


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034723701


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];

Review Comment:
   You can use the `partition()` method from `std::iter::Iterator ` which can split a iter/vec into two parts based on a true/false predicate, this is more FP style.
   
   https://doc.rust-lang.org/std/iter/trait.Iterator.html#method.partition
   
   



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1036384076


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,386 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
+                }
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
+                    }
                 }
 
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Window(window) => {
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&window.window_expr, &mut used_columns)?;
+                let window_columns = window
+                    .window_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(window_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*window.input).clone()),
+                    )?),
+                    None => (*window.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {

Review Comment:
   Yes, that was my understanding of the proposal -- however I think the merits of that change should be discussed in a different Issue / PR -- this one has gotten crazy already!



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035609444


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   > It existed originally. We can found it in tpch-q19.
   
   I think it's a point that we can improve it 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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035646064


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,359 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }

Review Comment:
   There is `break` to return the inner loop.   I think for join conditions: `on(a.id = b.id and a.id = b.id2) where b.id = 10`, we should be able to infer more equality predicates.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034954098


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(

Review Comment:
   @mingmwang look like alias can't be in groupby.
   
   sql 1999
   ```
   <group by clause> Function
   Specify a grouped table derived by the application of the <group by clause> to the result of the previously specified clause.
   Format
   <group by clause> ::=
          GROUP BY <grouping specification>
   <grouping specification> ::=
            <grouping column reference>
          | <rollup list>
          | <cube list>
          | <grouping sets list>
          | <grand total>
          | <concatenated grouping>
   ```
   
   pg:
   ```
   -- create
   CREATE TABLE EMPLOYEE (
     a INTEGER PRIMARY KEY,
     b INTEGER,
     c INTEGER
   );
   
   -- insert
   INSERT INTO EMPLOYEE VALUES (0001, 0001, 0001);
   INSERT INTO EMPLOYEE VALUES (0002, 0002, 0002);
   
   -- fetch 
   SELECT sum(b) FROM EMPLOYEE group by as c;
   
   ERROR:  syntax error at or near "as"```



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1331618851

   Could you please also modify the UT `optimize_plan()` method and let the rule run twice and see what will happen ?
   
   ```
       fn optimize_plan(plan: &LogicalPlan) -> LogicalPlan {
           let rule = FilterPushDown::new();
           rule.optimize(plan, &mut OptimizerConfig::new())
               .expect("failed to optimize plan")
       }
   ```
   
   I think some of the logic in the rule like 'extract_or_clauses_for_join()' a reasonable optimization, but if the optimizer rules  were applied more than once that duplicate Filters might be generated.
   


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1033240509


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -1194,16 +1105,15 @@ mod tests {
 
         // filter appears below Union
         let expected = "Union\
-        \n  SubqueryAlias: test2\
-        \n    Projection: test.a AS b\
-        \n      Filter: test.a = Int64(1)\
+        \n  Filter: b = Int64(1)\

Review Comment:
   When meet `SubqueryAlias`, it can't be pushdown.
   will fix in #4384 



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035613833


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,359 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
+            }
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::SubqueryAlias(subquery_alias) => {
+                let mut replace_map = HashMap::new();
+                for (i, field) in
+                    subquery_alias.input.schema().fields().iter().enumerate()
+                {
+                    replace_map.insert(
+                        subquery_alias
+                            .schema
+                            .fields()
+                            .get(i)
+                            .unwrap()
+                            .qualified_name(),
+                        Expr::Column(field.qualified_column()),
+                    );
                 }
+                let new_predicate =
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    subquery_alias.input.clone(),
+                )?);
+                child_plan.with_new_inputs(&[new_filter])?
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {
+                let mut new_scan_filters = scan.filters.clone();
+                let mut new_predicate = vec![];
+
+                let filter_predicates = utils::split_conjunction_owned(
+                    utils::cnf_rewrite(filter.predicate().clone()),
+                );
+
+                for filter_expr in &filter_predicates {
+                    let (preserve_filter_node, add_to_provider) =
+                        match scan.source.supports_filter_pushdown(filter_expr)? {
+                            TableProviderFilterPushDown::Unsupported => (true, false),
+                            TableProviderFilterPushDown::Inexact => (true, true),
+                            TableProviderFilterPushDown::Exact => (false, true),
+                        };
+                    if preserve_filter_node {
+                        new_predicate.push(filter_expr.clone());
+                    }
+                    if add_to_provider {
+                        // avoid reduplicated filter expr.
+                        if new_scan_filters.contains(filter_expr) {
+                            continue;
+                        }
+                        new_scan_filters.push(filter_expr.clone());
+                    }
+                }
 
-impl OptimizerRule for FilterPushDown {
-    fn name(&self) -> &str {
-        "filter_push_down"
-    }
+                let new_scan = LogicalPlan::TableScan(TableScan {
+                    source: scan.source.clone(),
+                    projection: scan.projection.clone(),
+                    projected_schema: scan.projected_schema.clone(),
+                    table_name: scan.table_name.clone(),
+                    filters: new_scan_filters,
+                    fetch: scan.fetch,
+                });
+
+                match conjunction(new_predicate) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_scan),
+                    )?),
+                    None => new_scan,
+                }
+            }
+            _ => plan.clone(),
+        };
 
-    fn optimize(
-        &self,
-        plan: &LogicalPlan,
-        _: &mut OptimizerConfig,
-    ) -> Result<LogicalPlan> {
-        optimize(plan, State::default())
+        utils::optimize_children(self, &new_plan, optimizer_config)

Review Comment:
   pattern is 
   
   ```
   LogicalJoin -> push, optimize children, return directly
   
   LogicalFilter
       SubPlan (Projection .....)
   
   -> push, then optimize children
   
   Other  ->  optimize children
   
   ```



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034454013


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]
                     })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                    .collect::<HashMap<_, _>>();
+
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
+
+                // optimize inner
+                from_plan(

Review Comment:
   I remember you have another PR and added the new method `with_new_inputs`. I think you can call the new method. The comment here is confusing, it does not invoke any optimization method, but just create new plan with new inputs. 



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035499731


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];

Review Comment:
   🥲 I don't know how to use `partition()` when meet result in closure 



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] ursabot commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1332666869

   Benchmark runs are scheduled for baseline = f2e2c296fa651469683243c6eea8afd65a3ef0ce and contender = 3fe542f2afcd5360edc9abb7ad1e8243b560a6b2. 3fe542f2afcd5360edc9abb7ad1e8243b560a6b2 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Skipped :warning: Benchmarking of arrow-datafusion-commits is not supported on ec2-t3-xlarge-us-east-2] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/bd5a5d042762410697063e60bfbc3b7a...91e8bdff37d34663a3c283eba24ffc55/)
   [Skipped :warning: Benchmarking of arrow-datafusion-commits is not supported on test-mac-arm] [test-mac-arm](https://conbench.ursa.dev/compare/runs/d3a1eb22793d4db58862cea99824bd4d...6c9f449254b44c219d235ccadec8d05c/)
   [Skipped :warning: Benchmarking of arrow-datafusion-commits is not supported on ursa-i9-9960x] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/7224843de8dd498aac8566fdc2683801...234909a1237e447eaeb97f537f468fd4/)
   [Skipped :warning: Benchmarking of arrow-datafusion-commits is not supported on ursa-thinkcentre-m75q] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/21fc134d9ac442279641aa192e85f690...8b60d13a4e2f4c55b74430dbc4171c9f/)
   Buildkite builds:
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] sarahyurick commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
sarahyurick commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1329844132

   > @ayushdg @sarahyurick It would be good to test the impact of this change in Dask SQL to make sure it doesn't cause any regressions for us.
   
   Looks good on my end - I'm seeing a performance improvement with the changes in this PR over DF 14.0.0


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034752803


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,386 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
+                }
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
+                    }
                 }
 
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Window(window) => {
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&window.window_expr, &mut used_columns)?;
+                let window_columns = window
+                    .window_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(window_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*window.input).clone()),
+                    )?),
+                    None => (*window.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {

Review Comment:
   I have a feeling that the data source/table scan filter push down should be handled in a physical planing phases, either by a physical rule or during converting LogicalPlan::TableScan to a physical Exec(ParquetExec).
   
   



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034602891


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]
                     })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                    .collect::<HashMap<_, _>>();
+
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
+
+                // optimize inner
+                from_plan(
+                    filter.input(),
+                    &(**filter.input()).expressions(),
+                    &[new_filter],
+                )?
             }
-
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let replace_map = input
+                        .schema()
+                        .fields()
+                        .iter()
+                        .map(|field| (field.qualified_name(), col(field.name())))
+                        .collect::<HashMap<_, _>>();
+
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {
+                let mut new_scan_filters = scan.filters.clone();
+                let mut new_predicate = vec![];
+
+                let filter_predicates = utils::split_conjunction_owned(
+                    utils::cnf_rewrite(filter.predicate().clone()),
+                );
+
+                for filter_expr in &filter_predicates {
+                    let (preserve_filter_node, add_to_provider) =
+                        match scan.source.supports_filter_pushdown(filter_expr)? {
+                            TableProviderFilterPushDown::Unsupported => (true, false),
+                            TableProviderFilterPushDown::Inexact => (true, true),
+                            TableProviderFilterPushDown::Exact => (false, true),
+                        };
+                    if preserve_filter_node {
+                        new_predicate.push(filter_expr.clone());
+                    }
+                    if add_to_provider {
+                        // avoid reduplicated filter expr.
+                        if new_scan_filters.contains(filter_expr) {
+                            continue;
+                        }
+                        new_scan_filters.push(filter_expr.clone());
+                    }
+                }
 
-impl OptimizerRule for FilterPushDown {
-    fn name(&self) -> &str {
-        "filter_push_down"
-    }
+                let new_scan = LogicalPlan::TableScan(TableScan {
+                    source: scan.source.clone(),
+                    projection: scan.projection.clone(),
+                    projected_schema: scan.projected_schema.clone(),
+                    table_name: scan.table_name.clone(),
+                    filters: new_scan_filters,
+                    fetch: scan.fetch,
+                });
+
+                match conjunction(new_predicate) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_scan),
+                    )?),
+                    None => new_scan,
+                }
+            }
+            _ => plan.clone(),
+        };
 

Review Comment:
   resolved



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034718656


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(

Review Comment:
   I do not get the logic of Filter push down to `Aggregate` here. Why does it need to check the `agg_columns`?  Should we just check `group exprs` here ? And group exprs could introduce `Alias` (group by a + b as c), need to construct a replace_map from c - > a + b.  And if the filter expr's columns can be replaced by the replace map, they should be push predicates.
   
   I think the logic here is not correct, some columns can be used/referenced in both agg exprs and group exprs.  



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1331632285

   Before this PR,  there is a global state which can help to avoid duplicate Filters been generated and pushed down.
   Now the global state is removed. Need to double conform the behavior is unchanged. 


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035611930


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,359 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
+            }
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::SubqueryAlias(subquery_alias) => {
+                let mut replace_map = HashMap::new();
+                for (i, field) in
+                    subquery_alias.input.schema().fields().iter().enumerate()
+                {
+                    replace_map.insert(
+                        subquery_alias
+                            .schema
+                            .fields()
+                            .get(i)
+                            .unwrap()
+                            .qualified_name(),
+                        Expr::Column(field.qualified_column()),
+                    );
                 }
+                let new_predicate =
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    subquery_alias.input.clone(),
+                )?);
+                child_plan.with_new_inputs(&[new_filter])?
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {
+                let mut new_scan_filters = scan.filters.clone();
+                let mut new_predicate = vec![];
+
+                let filter_predicates = utils::split_conjunction_owned(
+                    utils::cnf_rewrite(filter.predicate().clone()),
+                );
+
+                for filter_expr in &filter_predicates {
+                    let (preserve_filter_node, add_to_provider) =
+                        match scan.source.supports_filter_pushdown(filter_expr)? {
+                            TableProviderFilterPushDown::Unsupported => (true, false),
+                            TableProviderFilterPushDown::Inexact => (true, true),
+                            TableProviderFilterPushDown::Exact => (false, true),
+                        };
+                    if preserve_filter_node {
+                        new_predicate.push(filter_expr.clone());
+                    }
+                    if add_to_provider {
+                        // avoid reduplicated filter expr.
+                        if new_scan_filters.contains(filter_expr) {
+                            continue;
+                        }
+                        new_scan_filters.push(filter_expr.clone());
+                    }
+                }
 
-impl OptimizerRule for FilterPushDown {
-    fn name(&self) -> &str {
-        "filter_push_down"
-    }
+                let new_scan = LogicalPlan::TableScan(TableScan {
+                    source: scan.source.clone(),
+                    projection: scan.projection.clone(),
+                    projected_schema: scan.projected_schema.clone(),
+                    table_name: scan.table_name.clone(),
+                    filters: new_scan_filters,
+                    fetch: scan.fetch,
+                });
+
+                match conjunction(new_predicate) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_scan),
+                    )?),
+                    None => new_scan,
+                }
+            }
+            _ => plan.clone(),
+        };
 
-    fn optimize(
-        &self,
-        plan: &LogicalPlan,
-        _: &mut OptimizerConfig,
-    ) -> Result<LogicalPlan> {
-        optimize(plan, State::default())
+        utils::optimize_children(self, &new_plan, optimizer_config)

Review Comment:
   > I see there was an optimize_children() call at the beginning of the method.
   
   Beginning place that called `optimize_children()` return directly. So they don't intersect.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] ayushdg commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
ayushdg commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035907568


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,386 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
+                }
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
+                    }
                 }
 
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Window(window) => {
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&window.window_expr, &mut used_columns)?;
+                let window_columns = window
+                    .window_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(window_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*window.input).clone()),
+                    )?),
+                    None => (*window.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {

Review Comment:
   Is the suggestion here that the `LogicalPlan::TableScan` will no longer contain filters at the logical planning phase. In that case would this rule just generate filters that can be pushed down to be used by some followup rule? 
   (Apologies if the question doesn't make too much sense, I'm still not completely familiar with datafusion internals especially around physical plans). 



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034727674


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(

Review Comment:
   Same with above, It's from original code😂.
   
   I am also a little confused with these code. To avoid problems, I did not modify it.
   
   You remind me😂.
   
   As for `groupby`, I think original code forgot to handle it, it's a future ticket.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1331592683

   >  You can try this: select (a + b) as c, count(*) from Table_A group by 1
   
   ```rust
   #[test]
   fn push_down_filter_groupby_expr_contains_alias() {
       let sql = "SELECT (col_int32 + col_uint32) AS c, count(*) from test group by 1";
       let plan = test_sql(sql).unwrap();
       let expected = "Projection: test.col_int32 + test.col_uint32 AS c, COUNT(UInt8(1))\
       \n  Aggregate: groupBy=[[test.col_int32 + CAST(test.col_uint32 AS Int32)]], aggr=[[COUNT(UInt8(1))]]\
       \n    TableScan: test projection=[col_int32, col_uint32]";
       assert_eq!(expected, format!("{:?}", plan));
   }
   ```
   
   current implementation will put alias into projection expr


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1331756958

   All followup enhancement in #4433


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034284068


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down

Review Comment:
   Yes, I think we can.
   In fact, SEMI/ANTI JOIN is similar with FILTER
   Current code is just for refactor, I don't include extra enhancement.
   Related issue to track problems like this https://github.com/apache/arrow-datafusion/issues/4413



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034632791


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]

Review Comment:
   has resolved this bug, and add UT, thanks @mingmwang !
   
   The original UT `test_crossjoin_with_or_clause` can discover this bug  there,  but bypassed it with alias.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035650796


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,359 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }

Review Comment:
   > There is `break` to return the inner loop. I think for join conditions: `on(a.id = b.id and a.id = b.id2) where b.id = 10`, we should be able to infer more equality predicates.
   
   It should be common optimization, `infer conditon`, Many rule about `join` need to it.
   
   It should be a future ticket, and I think it's complex, because `infer condtion` sometime will cause bad effect



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034621165


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]

Review Comment:
   Yes, original code here exist bug.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035613833


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,359 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
+            }
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::SubqueryAlias(subquery_alias) => {
+                let mut replace_map = HashMap::new();
+                for (i, field) in
+                    subquery_alias.input.schema().fields().iter().enumerate()
+                {
+                    replace_map.insert(
+                        subquery_alias
+                            .schema
+                            .fields()
+                            .get(i)
+                            .unwrap()
+                            .qualified_name(),
+                        Expr::Column(field.qualified_column()),
+                    );
                 }
+                let new_predicate =
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    subquery_alias.input.clone(),
+                )?);
+                child_plan.with_new_inputs(&[new_filter])?
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {
+                let mut new_scan_filters = scan.filters.clone();
+                let mut new_predicate = vec![];
+
+                let filter_predicates = utils::split_conjunction_owned(
+                    utils::cnf_rewrite(filter.predicate().clone()),
+                );
+
+                for filter_expr in &filter_predicates {
+                    let (preserve_filter_node, add_to_provider) =
+                        match scan.source.supports_filter_pushdown(filter_expr)? {
+                            TableProviderFilterPushDown::Unsupported => (true, false),
+                            TableProviderFilterPushDown::Inexact => (true, true),
+                            TableProviderFilterPushDown::Exact => (false, true),
+                        };
+                    if preserve_filter_node {
+                        new_predicate.push(filter_expr.clone());
+                    }
+                    if add_to_provider {
+                        // avoid reduplicated filter expr.
+                        if new_scan_filters.contains(filter_expr) {
+                            continue;
+                        }
+                        new_scan_filters.push(filter_expr.clone());
+                    }
+                }
 
-impl OptimizerRule for FilterPushDown {
-    fn name(&self) -> &str {
-        "filter_push_down"
-    }
+                let new_scan = LogicalPlan::TableScan(TableScan {
+                    source: scan.source.clone(),
+                    projection: scan.projection.clone(),
+                    projected_schema: scan.projected_schema.clone(),
+                    table_name: scan.table_name.clone(),
+                    filters: new_scan_filters,
+                    fetch: scan.fetch,
+                });
+
+                match conjunction(new_predicate) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_scan),
+                    )?),
+                    None => new_scan,
+                }
+            }
+            _ => plan.clone(),
+        };
 
-    fn optimize(
-        &self,
-        plan: &LogicalPlan,
-        _: &mut OptimizerConfig,
-    ) -> Result<LogicalPlan> {
-        optimize(plan, State::default())
+        utils::optimize_children(self, &new_plan, optimizer_config)

Review Comment:
   pattern is 
   
   ```
   LogicalJoin -> push, optimize children, **return directly**
   LogicalFilter -> match, get the filter
   Other  ->  optimize children, **return directly**
   
   get filter
   
   match subPlan of filter {
   
   }
   
   ```



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035626623


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   I'm afraid they can not be removed from the original exprs.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034444658


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]

Review Comment:
   I understand this is coming from the existing logic, but could it encounter conflicting unqualified field names here?  For example a Projection contains both a.id, b.id



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034734115


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -926,8 +841,7 @@ mod tests {
             Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b) AS total_salary]]\

Review Comment:
   Please add one more case that the  Aggregate group exprs are complex exprs with Alias like (a + b) as c



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1032786851


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   This is a problem that existed originally, I found it in tpch-q19
   
   



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1330881348

   @mingmwang look like alias can't be in groupby.
   
   sql 1999
   ```
   <group by clause> Function
   Specify a grouped table derived by the application of the <group by clause> to the result of the previously specified clause.
   Format
   <group by clause> ::=
          GROUP BY <grouping specification>
   <grouping specification> ::=
            <grouping column reference>
          | <rollup list>
          | <cube list>
          | <grouping sets list>
          | <grand total>
          | <concatenated grouping>
   ```
   
   pg:
   ```
   -- create
   CREATE TABLE EMPLOYEE (
     a INTEGER PRIMARY KEY,
     b INTEGER,
     c INTEGER
   );
   
   -- insert
   INSERT INTO EMPLOYEE VALUES (0001, 0001, 0001);
   INSERT INTO EMPLOYEE VALUES (0002, 0002, 0002);
   
   -- fetch 
   SELECT sum(b) FROM EMPLOYEE group by as c;
   
   ERROR:  syntax error at or near "as"
   ```


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034277882


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down

Review Comment:
   Does the same logic can apply to LeftSemi and RightSemi join ?



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1330192696

   Why we use Arc to wrap all the inputs? Is it because of nested types and Rust does not support nested struct, must wrap the nested structs with Box or Arc ?


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034718656


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(

Review Comment:
   I do not get the logic of Filter push down to `Aggregate` here. Why does it need to check the `agg_columns`?  Should we just check `group exprs` here ? And group exprs could introduce `Alias` (group by a + b as c), need to construct a replace_map from c - > a + b 
   And if the filter expr's columns can be replaced by the replace map, they should be push predicates.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035650796


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,359 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }

Review Comment:
   > There is `break` to return the inner loop. I think for join conditions: `on(a.id = b.id and a.id = b.id2) where b.id = 10`, we should be able to infer more equality predicates.
   
   It should be common optimization, `infer conditon`, Many rule about `join` need to it.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034799939


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -926,8 +841,7 @@ mod tests {
             Aggregate: groupBy=[[test.a]], aggr=[[SUM(test.b) AS total_salary]]\

Review Comment:
   added `filter_complex_agg`



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] andygrove commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
andygrove commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1329272653

   @ayushdg @sarahyurick It would be good to test the impact of this change in Dask SQL to make sure it doesn't cause any regressions for us. 


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034454013


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]
                     })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                    .collect::<HashMap<_, _>>();
+
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
+
+                // optimize inner
+                from_plan(

Review Comment:
   I remember you have another PR and add the new method `with_new_inputs`. I think you can call the new method.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1330603626

   regression is fixed in #4384


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1328042121

   I need to set aside time to review this PR carefully given how important FilterPushdown is  -- I will try and find some in the upcoming week. 


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1328674915

   This week I have some time and I can take a closer look at this PR carefully.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1033240509


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -1194,16 +1105,15 @@ mod tests {
 
         // filter appears below Union
         let expected = "Union\
-        \n  SubqueryAlias: test2\
-        \n    Projection: test.a AS b\
-        \n      Filter: test.a = Int64(1)\
+        \n  Filter: b = Int64(1)\

Review Comment:
   will fix in #4384 



##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   It existed originally. We can found it in `q19`.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034632791


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]

Review Comment:
   has resolved this bug, and add UT, thanks @mingmwang !
   
   The original implementer looks like it knew there was a bug here and bypassed it with alias in UT. It's a strange behavior.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1330594493

   Some part of the `Filter` can also be pushed down through the` LogicalPlan::WIndow ` if the filter exprs are part of the partition by keys of the WIndow. The logic should be similar to `LogicalPlan::Aggregate` group by keys.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035098171


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,386 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
+                }
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
+                    }
                 }
 
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Window(window) => {
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&window.window_expr, &mut used_columns)?;
+                let window_columns = window
+                    .window_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(window_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*window.input).clone()),
+                    )?),
+                    None => (*window.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {

Review Comment:
   I agree that the pushdown into scan could be done as part of the physical planning phase. However, since the current filter pushdown happens in the logical planning phase I think it is ok to keep the same behavior in this PR and move the pushdown in some other PR



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034727674


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(

Review Comment:
   Same with above, It's from original code😂.I am also a little confused with these code. To avoid problems, I did not modify it.You remind me😂.
   
   As for `groupby`, I think original code forgot to handle it, it's a future ticket.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034444658


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]

Review Comment:
   I understand this is coming from the existing logic, but could it encounter conflict unqualified field names here?  



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
alamb commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034879272


##########
datafusion/core/tests/sql/joins.rs:
##########
@@ -1636,15 +1636,14 @@ async fn reduce_left_join_3() -> Result<()> {
             "Explain [plan_type:Utf8, plan:Utf8]",
             "  Projection: t3.t1_id, t3.t1_name, t3.t1_int, t2.t2_id, t2.t2_name, t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]",
             "    Left Join: t3.t1_int = t2.t2_int [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]",
-            "      Filter: t3.t1_id < UInt32(100) [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]",
-            "        SubqueryAlias: t3 [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]",
-            "          Inner Join: t1.t1_id = t2.t2_id [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]",
+            "      SubqueryAlias: t3 [t1_id:UInt32;N, t1_name:Utf8;N, t1_int:UInt32;N, t2_id:UInt32;N, t2_name:Utf8;N, t2_int:UInt32;N]",

Review Comment:
   this looks like a better plan as the filters have been pushed down through the join 👍 



##########
datafusion/optimizer/src/optimizer.rs:
##########
@@ -184,8 +184,9 @@ impl Optimizer {
             rules.push(Arc::new(FilterNullJoinKeys::default()));
         }
         rules.push(Arc::new(EliminateOuterJoin::new()));
-        rules.push(Arc::new(FilterPushDown::new()));
+        // Filter can't pushdown Limit, we should do PushDownFilter after LimitPushDown

Review Comment:
   ```suggestion
           // Filters can't be pushed down past Limits, we should do PushDownFilter after LimitPushDown
   ```
   
   Is this the intention of running Filter pushdown after limit pushdown?



##########
benchmarks/expected-plans/q7.txt:
##########
@@ -14,9 +14,9 @@ Sort: shipping.supp_nation ASC NULLS LAST, shipping.cust_nation ASC NULLS LAST,
                         TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate]
                     TableScan: orders projection=[o_orderkey, o_custkey]
                   TableScan: customer projection=[c_custkey, c_nationkey]
-                Filter: n1.n_name = Utf8("FRANCE") OR n1.n_name = Utf8("GERMANY")
-                  SubqueryAlias: n1
+                SubqueryAlias: n1

Review Comment:
   👍  these plan changes certainly look better to me



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1327097229

   The rule re-implementation of this rule was complex and now is completed.
   Now all that remains is an equally complex `ProjectionPushDown`.
   
   In fact, current `ProjectionPushDown` is a little wrong, it mix pushdown and column-prune, and it just prune bottom plannode ( tablescan ).


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035631504


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   I think carefully, yes, we can't remove it.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1032786229


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.
+    let or_to_left = extract_or_clauses_for_join(
+        &keep_predicates.iter().collect::<Vec<_>>(),
+        left.schema(),
+        left_preserved,
+    );
+    let or_to_right = extract_or_clauses_for_join(
+        &keep_predicates.iter().collect::<Vec<_>>(),
+        right.schema(),
+        right_preserved,
+    );
     let on_or_to_left = extract_or_clauses_for_join(
-        &on_to_keep.iter().collect::<Vec<_>>(),
+        &keep_condition.iter().collect::<Vec<_>>(),
         left.schema(),
         left_preserved,
     );
     let on_or_to_right = extract_or_clauses_for_join(
-        &on_to_keep.iter().collect::<Vec<_>>(),
+        &keep_condition.iter().collect::<Vec<_>>(),
         right.schema(),
         right_preserved,
     );
 
-    // Build new filter states using pushable predicates
-    // from current optimizer states and from ON clause.
-    // Then recursively call optimization for both join inputs
-    let mut left_state = State::default();
-    left_state.append_predicates(to_left);
-    left_state.append_predicates(on_to_left);
-    or_to_left
-        .0
-        .into_iter()
-        .zip(or_to_left.1)
-        .for_each(|(expr, cols)| left_state.filters.push((expr, cols)));
-    on_or_to_left
-        .0
-        .into_iter()
-        .zip(on_or_to_left.1)
-        .for_each(|(expr, cols)| left_state.filters.push((expr, cols)));
-    let left = optimize(left, left_state)?;

Review Comment:
   Original implementation here is bottom-up, outside is top down. Now unify to `top down` in all places.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1032786229


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.
+    let or_to_left = extract_or_clauses_for_join(
+        &keep_predicates.iter().collect::<Vec<_>>(),
+        left.schema(),
+        left_preserved,
+    );
+    let or_to_right = extract_or_clauses_for_join(
+        &keep_predicates.iter().collect::<Vec<_>>(),
+        right.schema(),
+        right_preserved,
+    );
     let on_or_to_left = extract_or_clauses_for_join(
-        &on_to_keep.iter().collect::<Vec<_>>(),
+        &keep_condition.iter().collect::<Vec<_>>(),
         left.schema(),
         left_preserved,
     );
     let on_or_to_right = extract_or_clauses_for_join(
-        &on_to_keep.iter().collect::<Vec<_>>(),
+        &keep_condition.iter().collect::<Vec<_>>(),
         right.schema(),
         right_preserved,
     );
 
-    // Build new filter states using pushable predicates
-    // from current optimizer states and from ON clause.
-    // Then recursively call optimization for both join inputs
-    let mut left_state = State::default();
-    left_state.append_predicates(to_left);
-    left_state.append_predicates(on_to_left);
-    or_to_left
-        .0
-        .into_iter()
-        .zip(or_to_left.1)
-        .for_each(|(expr, cols)| left_state.filters.push((expr, cols)));
-    on_or_to_left
-        .0
-        .into_iter()
-        .zip(on_or_to_left.1)
-        .for_each(|(expr, cols)| left_state.filters.push((expr, cols)));
-    let left = optimize(left, left_state)?;

Review Comment:
   Original implementation here is bottom-up, outside is top down. Now unify to `top down`.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034473069


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]
                     })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                    .collect::<HashMap<_, _>>();
+
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
+
+                // optimize inner
+                from_plan(
+                    filter.input(),
+                    &(**filter.input()).expressions(),
+                    &[new_filter],
+                )?
             }
-
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let replace_map = input
+                        .schema()
+                        .fields()
+                        .iter()
+                        .map(|field| (field.qualified_name(), col(field.name())))
+                        .collect::<HashMap<_, _>>();
+
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,

Review Comment:
   Looks like the `replace_map` for Union is not correct. My understanding for Union is we always use the left side field names as the output schema of Union. If we try to push down Filter to union's children,  need to replace the column in the Filter with the field name of each union's input.  The mapping should be based on the field index.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] alamb commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
alamb commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1332663517

   I think this PR has been outstanding enough and has had enough feedback and testing. Let's plan on completing the follow on work as other PRs. Thank you so much @jackwener and @mingmwang  for your careful reviews and @sarahyurick  for your testing 


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034284068


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down

Review Comment:
   Yes, I think we can.
   Related issue to track problems like this https://github.com/apache/arrow-datafusion/issues/4413



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034743301


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(

Review Comment:
   You are re-implementing the rule, do not leave it for a future ticket.  The similar logic can be applied to Window partition by exprs.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1331709722

   Except for the LogicalPlan::Window, the others LGTM.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035603648


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   Is it still a TODO here ?



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035609444


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   > It existed originally. We can found it in tpch-q19.
   
   Yes, I just add a comment, because I think it's a point that we can improve it 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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035499731


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,336 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];

Review Comment:
   🥲 I don't know how to use `partition()` when `Result` happen in closure 



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1032786851


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   This is a problem that existed originally, I found it in tpch-q19
   
   



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034284043


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down

Review Comment:
   And for Out joins:
   For Left Out join, if we have a query `SELECT t1_id, t1_name, t2_name FROM t1 LEFT JOIN t2 ON (t1_id = t2_id and t2_id >= 100);` ` t2_id >= 100 ` can be pushed down to the right side.
   
   For Right Out join, if we have a query `SELECT t1_id, t1_name, t2_name FROM t1 Right JOIN t2 ON (t1_id = t2_id and t1_id >= 100);` ` t1_id >= 100 ` can be pushed down to the left side.
   
   
   



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1331584666

   > @mingmwang look like alias can't be in groupby.
   > 
   > sql 1999
   > 
   > ```
   > <group by clause> Function
   > Specify a grouped table derived by the application of the <group by clause> to the result of the previously specified clause.
   > Format
   > <group by clause> ::=
   >        GROUP BY <grouping specification>
   > <grouping specification> ::=
   >          <grouping column reference>
   >        | <rollup list>
   >        | <cube list>
   >        | <grouping sets list>
   >        | <grand total>
   >        | <concatenated grouping>
   > ```
   > 
   > pg:
   > 
   > ```
   > -- create
   > CREATE TABLE EMPLOYEE (
   >   a INTEGER PRIMARY KEY,
   >   b INTEGER,
   >   c INTEGER
   > );
   > 
   > -- insert
   > INSERT INTO EMPLOYEE VALUES (0001, 0001, 0001);
   > INSERT INTO EMPLOYEE VALUES (0002, 0002, 0002);
   > 
   > -- fetch 
   > SELECT sum(b) FROM EMPLOYEE group by a as hh;
   > 
   > ERROR:  syntax error at or near "as"
   > ```
   
   You can try this:
   
   `select (a + b) as c, count(*) from Table_A group by 1`
   


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1330186642

    It seems that with the LogicalPlan structs, it is not that convenient to write pattern-matching and destructuring code, because of the Arc.
   
   ````
   match plan {
     LogicalPlan::Filter(Filter{predicate, LogicalPlan::Sort(sort)}) => {
     }
     LogicalPlan::Filter(Filter{predicate, LogicalPlan::Projection(proj)}) =>{
     }
     LogicalPlan::Filter(Filter{predicate, LogicalPlan::Aggregate(agg)}) =>{
     }
   }
   ````


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034560645


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]
                     })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                    .collect::<HashMap<_, _>>();
+
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
+
+                // optimize inner
+                from_plan(
+                    filter.input(),
+                    &(**filter.input()).expressions(),
+                    &[new_filter],
+                )?
             }
-
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let replace_map = input
+                        .schema()
+                        .fields()
+                        .iter()
+                        .map(|field| (field.qualified_name(), col(field.name())))
+                        .collect::<HashMap<_, _>>();
+
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,

Review Comment:
   My mistake, fixed it.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1331609471

   > > You can try this: select (a + b) as c, count(*) from Table_A group by 1
   > 
   > ```rust
   > #[test]
   > fn push_down_filter_groupby_expr_contains_alias() {
   >     let sql = "SELECT (col_int32 + col_uint32) AS c, count(*) from test group by 1";
   >     let plan = test_sql(sql).unwrap();
   >     let expected = "Projection: test.col_int32 + test.col_uint32 AS c, COUNT(UInt8(1))\
   >     \n  Aggregate: groupBy=[[test.col_int32 + CAST(test.col_uint32 AS Int32)]], aggr=[[COUNT(UInt8(1))]]\
   >     \n    TableScan: test projection=[col_int32, col_uint32]";
   >     assert_eq!(expected, format!("{:?}", plan));
   > }
   > ```
   > 
   > current implementation will put alias into projection expr
   > 
   > run in integration test. exist bug #4430, run it need to remove bug rule.
   > 
   > ```rust
   > #[test]
   > #[ignore]
   > // TODO: UnwrapCastInComparison exist bug.
   > fn push_down_filter_groupby_expr_contains_alias() {
   >     let sql = "SELECT * FROM (SELECT (col_int32 + col_uint32) AS c, count(*) FROM test GROUP BY 1) where c > 3";
   >     let plan = test_sql(sql).unwrap();
   >     let expected = "Projection: c, COUNT(UInt8(1))\
   >     \n  Projection: test.col_int32 + test.col_uint32 AS c, COUNT(UInt8(1))\
   >     \n    Aggregate: groupBy=[[test.col_int32 + CAST(test.col_uint32 AS Int32)]], aggr=[[COUNT(UInt8(1))]]\
   >     \n      Filter: CAST(test.col_int32 + test.col_uint32 AS Int64) > Int64(3)\
   >     \n        TableScan: test projection=[col_int32, col_uint32]";
   >     assert_eq!(expected, format!("{:?}", plan));
   > }
   > ```
   
   Ok, got it.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] mingmwang commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
mingmwang commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1035609575


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,359 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
+        let child_plan = &**filter.input();
+        let new_plan = match child_plan {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
+            }
+            LogicalPlan::Repartition(_)
+            | LogicalPlan::Distinct(_)
+            | LogicalPlan::Sort(_) => {
+                // commutable
+                let new_filter =
+                    plan.with_new_inputs(&[
+                        (**(child_plan.inputs().get(0).unwrap())).clone()
+                    ])?;
+                child_plan.with_new_inputs(&[new_filter])?
+            }
+            LogicalPlan::SubqueryAlias(subquery_alias) => {
+                let mut replace_map = HashMap::new();
+                for (i, field) in
+                    subquery_alias.input.schema().fields().iter().enumerate()
+                {
+                    replace_map.insert(
+                        subquery_alias
+                            .schema
+                            .fields()
+                            .get(i)
+                            .unwrap()
+                            .qualified_name(),
+                        Expr::Column(field.qualified_column()),
+                    );
                 }
+                let new_predicate =
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    subquery_alias.input.clone(),
+                )?);
+                child_plan.with_new_inputs(&[new_filter])?
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
-                })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
+                    .enumerate()
+                    .map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        (field.qualified_name(), expr)
+                    })
+                    .collect::<HashMap<_, _>>();
 
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
+                // re-write all filters based on this projection
+                // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    replace_cols_by_name(filter.predicate().clone(), &replace_map)?,
+                    projection.input.clone(),
+                )?);
 
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
-                    })
-                    .collect::<Result<Vec<_>>>()?;
-                state.filters.extend(join_side_filters);
+                child_plan.with_new_inputs(&[new_filter])?
             }
+            LogicalPlan::Union(union) => {
+                let mut inputs = Vec::with_capacity(union.inputs.len());
+                for input in &union.inputs {
+                    let mut replace_map = HashMap::new();
+                    for (i, field) in input.schema().fields().iter().enumerate() {
+                        replace_map.insert(
+                            union.schema.fields().get(i).unwrap().qualified_name(),
+                            Expr::Column(field.qualified_column()),
+                        );
+                    }
 
-            optimize_join(state, plan, left, right, on_filters)
-        }
-        LogicalPlan::TableScan(TableScan {
-            source,
-            projected_schema,
-            filters,
-            projection,
-            table_name,
-            fetch,
-        }) => {
-            let mut used_columns = HashSet::new();
-            let mut new_filters = filters.clone();
-
-            for (filter_expr, cols) in &state.filters {
-                let (preserve_filter_node, add_to_provider) =
-                    match source.supports_filter_pushdown(filter_expr)? {
-                        TableProviderFilterPushDown::Unsupported => (true, false),
-                        TableProviderFilterPushDown::Inexact => (true, true),
-                        TableProviderFilterPushDown::Exact => (false, true),
-                    };
-
-                if preserve_filter_node {
-                    used_columns.extend(cols.clone());
+                    let push_predicate =
+                        replace_cols_by_name(filter.predicate().clone(), &replace_map)?;
+                    inputs.push(Arc::new(LogicalPlan::Filter(Filter::try_new(
+                        push_predicate,
+                        input.clone(),
+                    )?)))
                 }
-
-                if add_to_provider {
-                    // Don't add expression again if it's already present in
-                    // pushed down filters.
-                    if new_filters.contains(filter_expr) {
-                        continue;
+                LogicalPlan::Union(Union {
+                    inputs,
+                    schema: plan.schema().clone(),
+                })
+            }
+            LogicalPlan::Aggregate(agg) => {
+                // An aggregate's aggregate columns are _not_ filter-commutable => collect these:
+                // * columns whose aggregation expression depends on
+                // * the aggregation columns themselves
+
+                // construct set of columns that `aggr_expr` depends on
+                let mut used_columns = HashSet::new();
+                exprlist_to_columns(&agg.aggr_expr, &mut used_columns)?;
+                let agg_columns = agg
+                    .aggr_expr
+                    .iter()
+                    .map(|x| Ok(Column::from_name(x.display_name()?)))
+                    .collect::<Result<HashSet<_>>>()?;
+                used_columns.extend(agg_columns);
+
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
+
+                let mut keep_predicates = vec![];
+                let mut push_predicates = vec![];
+                for expr in predicates {
+                    let columns = expr.to_columns()?;
+                    if columns.is_empty()
+                        || !columns
+                            .intersection(&used_columns)
+                            .collect::<HashSet<_>>()
+                            .is_empty()
+                    {
+                        keep_predicates.push(expr);
+                    } else {
+                        push_predicates.push(expr);
                     }
-                    new_filters.push(filter_expr.clone());
+                }
+
+                let child = match conjunction(push_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new((*agg.input).clone()),
+                    )?),
+                    None => (*agg.input).clone(),
+                };
+                let new_agg = from_plan(
+                    filter.input(),
+                    &filter.input().expressions(),
+                    &vec![child],
+                )?;
+                match conjunction(keep_predicates) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_agg),
+                    )?),
+                    None => new_agg,
+                }
+            }
+            LogicalPlan::Join(join) => {
+                match push_down_join(filter.input(), join, Some(filter.predicate()))? {
+                    Some(optimized_plan) => optimized_plan,
+                    None => plan.clone(),
                 }
             }
+            LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
+                let predicates = utils::split_conjunction_owned(utils::cnf_rewrite(
+                    filter.predicate().clone(),
+                ));
 
-            issue_filters(
-                state,
-                used_columns,
-                &LogicalPlan::TableScan(TableScan {
-                    source: source.clone(),
-                    projection: projection.clone(),
-                    projected_schema: projected_schema.clone(),
-                    table_name: table_name.clone(),
-                    filters: new_filters,
-                    fetch: *fetch,
-                }),
-            )
-        }
-        _ => {
-            // all other plans are _not_ filter-commutable
-            let used_columns = plan
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-    }
-}
+                push_down_all_join(predicates, filter.input(), left, right, vec![])?
+            }
+            LogicalPlan::TableScan(scan) => {
+                let mut new_scan_filters = scan.filters.clone();
+                let mut new_predicate = vec![];
+
+                let filter_predicates = utils::split_conjunction_owned(
+                    utils::cnf_rewrite(filter.predicate().clone()),
+                );
+
+                for filter_expr in &filter_predicates {
+                    let (preserve_filter_node, add_to_provider) =
+                        match scan.source.supports_filter_pushdown(filter_expr)? {
+                            TableProviderFilterPushDown::Unsupported => (true, false),
+                            TableProviderFilterPushDown::Inexact => (true, true),
+                            TableProviderFilterPushDown::Exact => (false, true),
+                        };
+                    if preserve_filter_node {
+                        new_predicate.push(filter_expr.clone());
+                    }
+                    if add_to_provider {
+                        // avoid reduplicated filter expr.
+                        if new_scan_filters.contains(filter_expr) {
+                            continue;
+                        }
+                        new_scan_filters.push(filter_expr.clone());
+                    }
+                }
 
-impl OptimizerRule for FilterPushDown {
-    fn name(&self) -> &str {
-        "filter_push_down"
-    }
+                let new_scan = LogicalPlan::TableScan(TableScan {
+                    source: scan.source.clone(),
+                    projection: scan.projection.clone(),
+                    projected_schema: scan.projected_schema.clone(),
+                    table_name: scan.table_name.clone(),
+                    filters: new_scan_filters,
+                    fetch: scan.fetch,
+                });
+
+                match conjunction(new_predicate) {
+                    Some(predicate) => LogicalPlan::Filter(Filter::try_new(
+                        predicate,
+                        Arc::new(new_scan),
+                    )?),
+                    None => new_scan,
+                }
+            }
+            _ => plan.clone(),
+        };
 
-    fn optimize(
-        &self,
-        plan: &LogicalPlan,
-        _: &mut OptimizerConfig,
-    ) -> Result<LogicalPlan> {
-        optimize(plan, State::default())
+        utils::optimize_children(self, &new_plan, optimizer_config)

Review Comment:
   I see there was an `optimize_children()` call at the beginning of the method. 
   Could you please explain why make a call to `optimize_children()` here again ?
   
   ````
   let filter = match plan {
               LogicalPlan::Filter(filter) => filter,
               // we also need to pushdown filter in Join.
               LogicalPlan::Join(join) => {
                   let optimized_plan = push_down_join(plan, join, None)?;
                   return match optimized_plan {
                       Some(optimized_plan) => {
                           utils::optimize_children(self, &optimized_plan, optimizer_config)
                       }
                       None => utils::optimize_children(self, plan, optimizer_config),
                   };
               }
               _ => return utils::optimize_children(self, plan, optimizer_config),
           };
   
   ````



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1033241313


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -403,94 +294,90 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet<Column>) -> Option<Ex
     predicate
 }
 
-fn optimize_join(
-    mut state: State,
+// push down join/cross-join
+fn push_down_all_join(
+    predicates: Vec<Expr>,
     plan: &LogicalPlan,
     left: &LogicalPlan,
     right: &LogicalPlan,
-    on_filter: Vec<Predicate>,
+    on_filter: Vec<Expr>,
 ) -> Result<LogicalPlan> {
+    let on_filter_empty = on_filter.is_empty();
     // Get pushable predicates from current optimizer state
     let (left_preserved, right_preserved) = lr_is_preserved(plan)?;
-    let to_left =
-        get_pushable_join_predicates(&state.filters, left.schema(), left_preserved);
-    let to_right =
-        get_pushable_join_predicates(&state.filters, right.schema(), right_preserved);
-    let to_keep: Predicates = state
-        .filters
-        .iter()
-        .filter(|(e, _)| !to_left.0.contains(&e) && !to_right.0.contains(&e))
-        .map(|(a, b)| (a, b))
-        .unzip();
+    let mut left_push = vec![];
+    let mut right_push = vec![];
+
+    let mut keep_predicates = vec![];
+    for predicate in predicates {
+        if left_preserved && can_pushdown_join_predicate(&predicate, left.schema())? {
+            left_push.push(predicate);
+        } else if right_preserved
+            && can_pushdown_join_predicate(&predicate, right.schema())?
+        {
+            right_push.push(predicate);
+        } else {
+            keep_predicates.push(predicate);
+        }
+    }
 
-    // Get pushable predicates from join filter
-    let (on_to_left, on_to_right, on_to_keep) = if on_filter.is_empty() {
-        ((vec![], vec![]), (vec![], vec![]), vec![])
-    } else {
+    let mut keep_condition = vec![];
+    if !on_filter.is_empty() {
         let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(plan)?;
-        let on_to_left =
-            get_pushable_join_predicates(&on_filter, left.schema(), on_left_preserved);
-        let on_to_right =
-            get_pushable_join_predicates(&on_filter, right.schema(), on_right_preserved);
-        let on_to_keep = on_filter
-            .iter()
-            .filter(|(e, _)| !on_to_left.0.contains(&e) && !on_to_right.0.contains(&e))
-            .map(|(a, _)| a.clone())
-            .collect::<Vec<_>>();
-
-        (on_to_left, on_to_right, on_to_keep)
-    };
+        for on in on_filter {
+            if on_left_preserved && can_pushdown_join_predicate(&on, left.schema())? {
+                left_push.push(on)
+            } else if on_right_preserved
+                && can_pushdown_join_predicate(&on, right.schema())?
+            {
+                right_push.push(on)
+            } else {
+                keep_condition.push(on)
+            }
+        }
+    }
 
     // Extract from OR clause, generate new predicates for both side of join if possible.
     // We only track the unpushable predicates above.
-    let or_to_left =
-        extract_or_clauses_for_join(&to_keep.0, left.schema(), left_preserved);
-    let or_to_right =
-        extract_or_clauses_for_join(&to_keep.0, right.schema(), right_preserved);
+    // TODO: we just get, but don't remove them from origin expr.

Review Comment:
   It existed originally. We can found it in `tpch-q19`.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on a diff in pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on code in PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#discussion_r1034632791


##########
datafusion/optimizer/src/push_down_filter.rs:
##########
@@ -500,302 +387,344 @@ fn optimize_join(
     //      vector will contain only join keys (without additional
     //      element representing filter).
     let expr = plan.expressions();
-    let expr = if !on_filter.is_empty() && on_to_keep.is_empty() {
+    let expr = if !on_filter_empty && keep_condition.is_empty() {
         // New filter expression is None - should remove last element
         expr[..expr.len() - 1].to_vec()
-    } else if !on_to_keep.is_empty() {
+    } else if !keep_condition.is_empty() {
         // Replace last element with new filter expression
         expr[..expr.len() - 1]
             .iter()
             .cloned()
-            .chain(once(on_to_keep.into_iter().reduce(Expr::and).unwrap()))
+            .chain(once(keep_condition.into_iter().reduce(Expr::and).unwrap()))
             .collect()
     } else {
         plan.expressions()
     };
     let plan = from_plan(plan, &expr, &[left, right])?;
 
-    if to_keep.0.is_empty() {
+    if keep_predicates.is_empty() {
         Ok(plan)
     } else {
         // wrap the join on the filter whose predicates must be kept
-        let plan = utils::add_filter(plan, &to_keep.0)?;
-        state.filters = remove_filters(&state.filters, &to_keep.1);
-
-        Ok(plan)
+        match conjunction(keep_predicates) {
+            Some(predicate) => Ok(LogicalPlan::Filter(Filter::try_new(
+                predicate,
+                Arc::new(plan),
+            )?)),
+            None => Ok(plan),
+        }
     }
 }
 
-fn optimize(plan: &LogicalPlan, mut state: State) -> Result<LogicalPlan> {
-    match plan {
-        LogicalPlan::Explain { .. } => {
-            // push the optimization to the plan of this explain
-            push_down(&state, plan)
-        }
-        LogicalPlan::Analyze { .. } => push_down(&state, plan),
-        LogicalPlan::Filter(filter) => {
-            let predicate = utils::cnf_rewrite(filter.predicate().clone());
-
-            utils::split_conjunction_owned(predicate)
-                .into_iter()
-                .try_for_each::<_, Result<()>>(|predicate| {
-                    let columns = predicate.to_columns()?;
-                    state.filters.push((predicate, columns));
-                    Ok(())
-                })?;
-
-            optimize(filter.input(), state)
+fn push_down_join(
+    plan: &LogicalPlan,
+    join: &Join,
+    parent_predicate: Option<&Expr>,
+) -> Result<Option<LogicalPlan>> {
+    let mut predicates = match parent_predicate {
+        Some(parent_predicate) => {
+            utils::split_conjunction_owned(utils::cnf_rewrite(parent_predicate.clone()))
         }
-        LogicalPlan::Projection(Projection {
-            input,
-            expr,
-            schema,
-        }) => {
-            // A projection is filter-commutable, but re-writes all predicate expressions
-            // collect projection.
-            let projection = schema
-                .fields()
-                .iter()
-                .enumerate()
-                .flat_map(|(i, field)| {
-                    // strip alias, as they should not be part of filters
-                    let expr = match &expr[i] {
-                        Expr::Alias(expr, _) => expr.as_ref().clone(),
-                        expr => expr.clone(),
+        None => vec![],
+    };
+
+    // Convert JOIN ON predicate to Predicates
+    let on_filters = join
+        .filter
+        .as_ref()
+        .map(|e| utils::split_conjunction_owned(e.clone()))
+        .unwrap_or_else(Vec::new);
+
+    if join.join_type == JoinType::Inner {
+        // For inner joins, duplicate filters for joined columns so filters can be pushed down
+        // to both sides. Take the following query as an example:
+        //
+        // ```sql
+        // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
+        // ```
+        //
+        // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
+        // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
+        //
+        // Join clauses with `Using` constraints also take advantage of this logic to make sure
+        // predicates reference the shared join columns are pushed to both sides.
+        // This logic should also been applied to conditions in JOIN ON clause
+        let join_side_filters = predicates
+            .iter()
+            .chain(on_filters.iter())
+            .filter_map(|predicate| {
+                let mut join_cols_to_replace = HashMap::new();
+                let columns = match predicate.to_columns() {
+                    Ok(columns) => columns,
+                    Err(e) => return Some(Err(e)),
+                };
+
+                for col in columns.iter() {
+                    for (l, r) in join.on.iter() {
+                        if col == l {
+                            join_cols_to_replace.insert(col, r);
+                            break;
+                        } else if col == r {
+                            join_cols_to_replace.insert(col, l);
+                            break;
+                        }
+                    }
+                }
+
+                if join_cols_to_replace.is_empty() {
+                    return None;
+                }
+
+                let join_side_predicate =
+                    match replace_col(predicate.clone(), &join_cols_to_replace) {
+                        Ok(p) => p,
+                        Err(e) => {
+                            return Some(Err(e));
+                        }
                     };
 
-                    // Convert both qualified and unqualified fields
-                    [
-                        (field.name().clone(), expr.clone()),
-                        (field.qualified_name(), expr),
-                    ]
-                })
-                .collect::<HashMap<_, _>>();
+                Some(Ok(join_side_predicate))
+            })
+            .collect::<Result<Vec<_>>>()?;
+        predicates.extend(join_side_filters);
+    }
+    if on_filters.is_empty() && predicates.is_empty() {
+        return Ok(None);
+    }
+    Ok(Some(push_down_all_join(
+        predicates,
+        plan,
+        &join.left,
+        &join.right,
+        on_filters,
+    )?))
+}
 
-            // re-write all filters based on this projection
-            // E.g. in `Filter: b\n  Projection: a > 1 as b`, we can swap them, but the filter must be "a > 1"
-            for (predicate, columns) in state.filters.iter_mut() {
-                *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
+impl OptimizerRule for PushDownFilter {
+    fn name(&self) -> &str {
+        "push_down_filter"
+    }
 
-                columns.clear();
-                expr_to_columns(predicate, columns)?;
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        optimizer_config: &mut OptimizerConfig,
+    ) -> Result<LogicalPlan> {
+        let filter = match plan {
+            LogicalPlan::Filter(filter) => filter,
+            // we also need to pushdown filter in Join.
+            LogicalPlan::Join(join) => {
+                let optimized_plan = push_down_join(plan, join, None)?;
+                return match optimized_plan {
+                    Some(optimized_plan) => {
+                        utils::optimize_children(self, &optimized_plan, optimizer_config)
+                    }
+                    None => utils::optimize_children(self, plan, optimizer_config),
+                };
             }
+            _ => return utils::optimize_children(self, plan, optimizer_config),
+        };
 
-            // optimize inner
-            let new_input = optimize(input, state)?;
-            Ok(from_plan(plan, expr, &[new_input])?)
-        }
-        LogicalPlan::Aggregate(Aggregate { aggr_expr, .. }) => {
-            // An aggregate's aggreagate columns are _not_ filter-commutable => collect these:
-            // * columns whose aggregation expression depends on
-            // * the aggregation columns themselves
-
-            // construct set of columns that `aggr_expr` depends on
-            let mut used_columns = HashSet::new();
-            exprlist_to_columns(aggr_expr, &mut used_columns)?;
-
-            let agg_columns = aggr_expr
-                .iter()
-                .map(|x| Ok(Column::from_name(x.display_name()?)))
-                .collect::<Result<HashSet<_>>>()?;
-            used_columns.extend(agg_columns);
-
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::Sort { .. } => {
-            // sort is filter-commutable
-            push_down(&state, plan)
-        }
-        LogicalPlan::Union(Union { inputs: _, schema }) => {
-            // union changing all qualifiers while building logical plan so we need
-            // to rewrite filters to push unqualified columns to inputs
-            let projection = schema
-                .fields()
-                .iter()
-                .map(|field| (field.qualified_name(), col(field.name())))
-                .collect::<HashMap<_, _>>();
-
-            // rewriting predicate expressions using unqualified names as replacements
-            if !projection.is_empty() {
-                for (predicate, columns) in state.filters.iter_mut() {
-                    *predicate = replace_cols_by_name(predicate.clone(), &projection)?;
-
-                    columns.clear();
-                    expr_to_columns(predicate, columns)?;
-                }
+        let new_plan = match &**filter.input() {
+            LogicalPlan::Filter(child_filter) => {
+                let new_predicate =
+                    and(filter.predicate().clone(), child_filter.predicate().clone());
+                let new_plan = LogicalPlan::Filter(Filter::try_new(
+                    new_predicate,
+                    child_filter.input().clone(),
+                )?);
+                return self.optimize(&new_plan, optimizer_config);
             }
-
-            push_down(&state, plan)
-        }
-        LogicalPlan::Limit(Limit { input, .. }) => {
-            // limit is _not_ filter-commutable => collect all columns from its input
-            let used_columns = input
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.qualified_column())
-                .collect::<HashSet<_>>();
-            issue_filters(state, used_columns, plan)
-        }
-        LogicalPlan::CrossJoin(CrossJoin { left, right, .. }) => {
-            optimize_join(state, plan, left, right, vec![])
-        }
-        LogicalPlan::Join(Join {
-            left,
-            right,
-            on,
-            filter,
-            join_type,
-            ..
-        }) => {
-            // Convert JOIN ON predicate to Predicates
-            let on_filters = filter
-                .as_ref()
-                .map(|e| {
-                    let predicates = utils::split_conjunction(e);
-
-                    predicates
-                        .into_iter()
-                        .map(|e| Ok((e.clone(), e.to_columns()?)))
-                        .collect::<Result<Vec<_>>>()
+            LogicalPlan::Sort(sort) => {
+                let new_filter = LogicalPlan::Filter(Filter::try_new(
+                    filter.predicate().clone(),
+                    sort.input.clone(),
+                )?);
+                LogicalPlan::Sort(Sort {
+                    expr: sort.expr.clone(),
+                    input: Arc::new(new_filter),
+                    fetch: sort.fetch,
                 })
-                .unwrap_or_else(|| Ok(vec![]))?;
-
-            if *join_type == JoinType::Inner {
-                // For inner joins, duplicate filters for joined columns so filters can be pushed down
-                // to both sides. Take the following query as an example:
-                //
-                // ```sql
-                // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1
-                // ```
-                //
-                // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while
-                // `t2.uid > 1` predicate needs to be pushed down to t2 table scan.
-                //
-                // Join clauses with `Using` constraints also take advantage of this logic to make sure
-                // predicates reference the shared join columns are pushed to both sides.
-                // This logic should also been applied to conditions in JOIN ON clause
-                let join_side_filters = state
-                    .filters
+            }
+            LogicalPlan::Projection(projection) => {
+                // A projection is filter-commutable, but re-writes all predicate expressions
+                // collect projection.
+                let replace_map = projection
+                    .schema
+                    .fields()
                     .iter()
-                    .chain(on_filters.iter())
-                    .filter_map(|(predicate, columns)| {
-                        let mut join_cols_to_replace = HashMap::new();
-                        for col in columns.iter() {
-                            for (l, r) in on {
-                                if col == l {
-                                    join_cols_to_replace.insert(col, r);
-                                    break;
-                                } else if col == r {
-                                    join_cols_to_replace.insert(col, l);
-                                    break;
-                                }
-                            }
-                        }
-
-                        if join_cols_to_replace.is_empty() {
-                            return None;
-                        }
-
-                        let join_side_predicate =
-                            match replace_col(predicate.clone(), &join_cols_to_replace) {
-                                Ok(p) => p,
-                                Err(e) => {
-                                    return Some(Err(e));
-                                }
-                            };
-
-                        let join_side_columns = columns
-                            .clone()
-                            .into_iter()
-                            // replace keys in join_cols_to_replace with values in resulting column
-                            // set
-                            .filter(|c| !join_cols_to_replace.contains_key(c))
-                            .chain(join_cols_to_replace.values().map(|v| (*v).clone()))
-                            .collect();
-
-                        Some(Ok((join_side_predicate, join_side_columns)))
+                    .enumerate()
+                    .flat_map(|(i, field)| {
+                        // strip alias, as they should not be part of filters
+                        let expr = match &projection.expr[i] {
+                            Expr::Alias(expr, _) => expr.as_ref().clone(),
+                            expr => expr.clone(),
+                        };
+
+                        // Convert both qualified and unqualified fields
+                        [
+                            (field.name().clone(), expr.clone()),
+                            (field.qualified_name(), expr),
+                        ]

Review Comment:
   has resolved this bug, and add UT, thanks @mingmwang !
   
   The original UT `test_crossjoin_with_or_clause` can discover this bug  there,  but it bypassed it with alias.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow-datafusion] jackwener commented on pull request #4365: reimplement `push_down_filter` to remove global-state

Posted by GitBox <gi...@apache.org>.
jackwener commented on PR #4365:
URL: https://github.com/apache/arrow-datafusion/pull/4365#issuecomment-1331646328

   Has added it in UT.


-- 
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: github-unsubscribe@arrow.apache.org

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