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 2021/05/28 18:46:53 UTC

[GitHub] [arrow-datafusion] jgoday opened a new pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

jgoday opened a new pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436


   # Which issue does this PR close?
   Closes #410.
   
    # Rationale for this change
   Uses boolean algebra laws to simplify binary boolean expressions.
   
   # What changes are included in this PR?
   New RemoveDuplicateFilters optimizer.
   
   # 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.

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r641908160



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,310 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::Expr;
+use crate::logical_plan::LogicalPlan;
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr { left, op: _, right } if left == right => simplify(left),

Review comment:
       I am seeing one other example from the test `1=1` is also a good example, it now simplifies to `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.

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643691688



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => x.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *right.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *right.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *left.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)

Review comment:
       Also here - is this still needed and doesn't this remove certain opportunities as the recursive call isn't done?




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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643404061



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {

Review comment:
       My mistake, should be fixed now.




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

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



[GitHub] [arrow-datafusion] Dandandan edited a comment on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan edited a comment on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-850792952


   Looks like a good start!
   
   I think it's good to think of these rules as simple rewrite rules we can apply to the expression tree.
   
   ```
   p OR p => p
   p AND p => p
   ```
   
   Some weeks ago I implemented some rules in this custom DataFusion optimizer (based on graph rewriting), you can see them here:
   
   https://github.com/Dandandan/datafusion-tokomak/blob/main/src/lib.rs#L44
   There are a lot of other patterns we can add if we do term rewriting in this way.
   
   Also, I think we should make this optimizer run on *any* `Expr`, not just those in `Filter` as this would benefit any `Expr` (in projections, aggregations, joins, etc)
   
   


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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643404456



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {

Review comment:
       You're right! should be fixed in latest commit




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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643417322



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)

Review comment:
       Changed, but I'm afraid this file docs will have to be fixed/completed later ...




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

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



[GitHub] [arrow-datafusion] Dandandan merged pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan merged pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436


   


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

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



[GitHub] [arrow-datafusion] codecov-commenter edited a comment on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-852434776


   # [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#436](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (af2c6ac) into [master](https://codecov.io/gh/apache/arrow-datafusion/commit/db4f098d38993b96ce1134c4bc7bf5c6579509cf?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (db4f098) will **increase** coverage by `0.98%`.
   > The diff coverage is `87.45%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-datafusion/pull/436/graphs/tree.svg?width=650&height=150&src=pr&token=JXwWBKD3D9&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #436      +/-   ##
   ==========================================
   + Coverage   74.94%   75.92%   +0.98%     
   ==========================================
     Files         146      154       +8     
     Lines       24314    26195    +1881     
   ==========================================
   + Hits        18221    19889    +1668     
   - Misses       6093     6306     +213     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [datafusion/src/execution/context.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvZXhlY3V0aW9uL2NvbnRleHQucnM=) | `92.08% <ø> (+0.03%)` | :arrow_up: |
   | [datafusion/src/optimizer/simplify\_expressions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvb3B0aW1pemVyL3NpbXBsaWZ5X2V4cHJlc3Npb25zLnJz) | `87.45% <87.45%> (ø)` | |
   | [datafusion-cli/src/print\_format.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi1jbGkvc3JjL3ByaW50X2Zvcm1hdC5ycw==) | `81.25% <0.00%> (-9.17%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/mod.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9tb2QucnM=) | `78.70% <0.00%> (-4.06%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/window\_functions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi93aW5kb3dfZnVuY3Rpb25zLnJz) | `85.71% <0.00%> (-3.01%)` | :arrow_down: |
   | [...tafusion/src/physical\_plan/datetime\_expressions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9kYXRldGltZV9leHByZXNzaW9ucy5ycw==) | `67.29% <0.00%> (-2.52%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/common.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9jb21tb24ucnM=) | `84.21% <0.00%> (-2.00%)` | :arrow_down: |
   | [datafusion/src/optimizer/hash\_build\_probe\_order.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvb3B0aW1pemVyL2hhc2hfYnVpbGRfcHJvYmVfb3JkZXIucnM=) | `58.53% <0.00%> (-1.97%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/repartition.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9yZXBhcnRpdGlvbi5ycw==) | `82.45% <0.00%> (-1.89%)` | :arrow_down: |
   | [ballista/rust/scheduler/src/planner.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9zY2hlZHVsZXIvc3JjL3BsYW5uZXIucnM=) | `66.91% <0.00%> (-0.74%)` | :arrow_down: |
   | ... and [57 more](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [db4f098...af2c6ac](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

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



[GitHub] [arrow-datafusion] alamb commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

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


   > (c > 5) OR ((d < 6) AND (c > 5)) should be simplified to (c > 5), isn't it?
   
   Yes you are correct -- I guess I was quickly trying to cook up examples similar to @Dandandan  that showed searching for an arbitrary instance of the same expression (even if it appeared in some nested structure) was going to incorrectly remove the nested instance in some 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.

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r641905765



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,310 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::Expr;
+use crate::logical_plan::LogicalPlan;
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr { left, op: _, right } if left == right => simplify(left),

Review comment:
       This seems not correct to me, it seems to "simplify" `a + a` to `a` too?




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

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



[GitHub] [arrow-datafusion] jgoday commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-854062163


   Thank you @Dandandan and @alamb  for all your help!
   
   


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r644922975



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),

Review comment:
       I merged it, but see this part might still be not 100% correct.
   
   if left and right are a column, the value can still be null (for some evaluations).
   Also the division by 0 (e.g. 0/0 or x/x where x is 0) is tricky.
   I think there is no easy way to do it safely.




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643689148



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)

Review comment:
       It seems if this matches any oppurtunity to go deeper is lost? Is this pattern still needed?




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

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



[GitHub] [arrow-datafusion] Dandandan commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-853977793


   Awesome thanks @jgoday 💯 


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

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



[GitHub] [arrow-datafusion] Dandandan edited a comment on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan edited a comment on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-850792952


   Looks like a good start!
   
   I think it's good to think of these rules as simple rewrite rules we can apply to the expression tree.
   
   ```
   p OR p => a
   p AND p => a
   ```
   
   Some weeks ago I implemented some rules in this custom DataFusion optimizer (based on graph rewriting), you can see them here:
   
   https://github.com/Dandandan/datafusion-tokomak/blob/main/src/lib.rs#L44
   There are a lot of other patterns we can add if we do term rewriting in this way.
   
   Also, I think we should make this optimizer run on *any* `Expr`, not just those in `Filter` as this would benefit any `Expr` (in projections, aggregations, joins, etc)
   
   


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r642666286



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)

Review comment:
       I think we can best use `b > 2 AND b > 2` here in the docs.
   Hopefully we can implement a better way of displaying expressions soon!




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

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



[GitHub] [arrow-datafusion] alamb commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643386412



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)

Review comment:
       Tracked in https://github.com/apache/arrow-datafusion/issues/347 for anyone following along :)




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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r641941522



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,310 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::Expr;
+use crate::logical_plan::LogicalPlan;
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr { left, op: _, right } if left == right => simplify(left),

Review comment:
       You're right, I was wrongly assuming that all expressions are boolean so ... my mistake.
   I have updated it. I hope I can check your other comments soon. Thanks !




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643854504



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)

Review comment:
       Ah it makes sense, I missed that `expr_contains` pattern matches on or/and 👍 




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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643450066



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {

Review comment:
       I like that last option :)




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

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



[GitHub] [arrow-datafusion] Dandandan merged pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan merged pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436


   


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

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



[GitHub] [arrow-datafusion] jgoday commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-850659787


   > (c > 5) OR ((d < 6) AND (c > 5) -- can't remove
   
   @alamb 
   I Have updated the unit tests, but
   (c > 5) OR ((d < 6) AND (c > 5)) should be simplified to (c > 5), isn't it?
   
   Maybe you mean a different expression ?
   
   
   
   


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r642668210



##########
File path: datafusion/src/optimizer/mod.rs
##########
@@ -25,4 +25,5 @@ pub mod hash_build_probe_order;
 pub mod limit_push_down;
 pub mod optimizer;
 pub mod projection_push_down;
+pub mod remove_duplicate_filters;

Review comment:
       Probably we should rename it now to something like "simplify expressions".




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

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



[GitHub] [arrow-datafusion] Dandandan commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-852936922


   It looks good to me @jgoday 
   Could you fix the clippy and formatting errors?


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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643412047



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false
+    }
+}
+
+fn is_zero<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(0))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 0. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 0. => true,
+        _ => false
+    }
+}
+
+fn is_true<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(left) => Expr::Negative(Box::new(simplify(right))),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if left == right => lit(0),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_zero(left) || is_zero(right) => lit(0),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right
+        } if left == right => lit(1),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Plus,
+            right
+        } if is_zero(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Plus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => x.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *right.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *right.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *left.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *right.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => x.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *left.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => x.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    match plan {
+        LogicalPlan::Filter { input, predicate } => Ok(LogicalPlan::Filter {
+            input: input.clone(),
+            predicate: simplify(predicate),
+        }),
+        LogicalPlan::Projection { expr, input, schema } => Ok(LogicalPlan::Projection {
+            expr: expr.into_iter().map(|x| simplify(x)).collect::<Vec<_>>(),
+            input: input.clone(),
+            schema: schema.clone(),
+        }),
+        _ => {
+            let new_inputs = plan
+                .inputs()
+                .iter()
+                .map(|input| optimize(input))
+                .collect::<Result<Vec<_>>>()?;
+
+            let expr = plan.expressions();

Review comment:
       Should this be valid ?
   
   ```
   fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
       let new_inputs = plan
           .inputs()
           .iter()
           .map(|input| optimize(input))
           .collect::<Result<Vec<_>>>()?;
       let expr = plan
           .expressions()
           .into_iter()
           .map(|x| simplify(&x))
           .collect::<Vec<_>>();
       utils::from_plan(&plan, &expr, &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.

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r642667853



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false
+    }
+}
+
+fn is_zero<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(0))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 0. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 0. => true,
+        _ => false
+    }
+}
+
+fn is_true<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(left) => Expr::Negative(Box::new(simplify(right))),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if left == right => lit(0),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_zero(left) || is_zero(right) => lit(0),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right
+        } if left == right => lit(1),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Plus,
+            right
+        } if is_zero(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Plus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => x.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *right.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *right.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *left.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *right.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => x.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *left.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => x.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    match plan {
+        LogicalPlan::Filter { input, predicate } => Ok(LogicalPlan::Filter {
+            input: input.clone(),
+            predicate: simplify(predicate),
+        }),
+        LogicalPlan::Projection { expr, input, schema } => Ok(LogicalPlan::Projection {
+            expr: expr.into_iter().map(|x| simplify(x)).collect::<Vec<_>>(),
+            input: input.clone(),
+            schema: schema.clone(),
+        }),
+        _ => {
+            let new_inputs = plan
+                .inputs()
+                .iter()
+                .map(|input| optimize(input))
+                .collect::<Result<Vec<_>>>()?;
+
+            let expr = plan.expressions();

Review comment:
       This could use something like `plan.expressions().into_iter().map(|x| simplify(x)).collect::<Vec<_>>()` ? and the other patterns could be removed here? (So it also applies for any node containing expressions.




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r642667063



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {

Review comment:
       Operator is probably `Copy` so no need to add `&`




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643439659



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {

Review comment:
       Or it might be better to just use `|` patterns (without `matches!`) in the code.




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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643415079



##########
File path: datafusion/src/optimizer/mod.rs
##########
@@ -25,4 +25,5 @@ pub mod hash_build_probe_order;
 pub mod limit_push_down;
 pub mod optimizer;
 pub mod projection_push_down;
+pub mod remove_duplicate_filters;

Review comment:
       Should we rename it to 'optimizer::simplify_expressions' ?




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643438683



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {

Review comment:
       A `matches!` returns a boolean so you can also use `||` / boolean or to cover the float cases and keep the rest in a single `matches!`




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

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



[GitHub] [arrow-datafusion] codecov-commenter edited a comment on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-852434776


   # [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#436](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (af2c6ac) into [master](https://codecov.io/gh/apache/arrow-datafusion/commit/db4f098d38993b96ce1134c4bc7bf5c6579509cf?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (db4f098) will **increase** coverage by `0.98%`.
   > The diff coverage is `87.45%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-datafusion/pull/436/graphs/tree.svg?width=650&height=150&src=pr&token=JXwWBKD3D9&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #436      +/-   ##
   ==========================================
   + Coverage   74.94%   75.92%   +0.98%     
   ==========================================
     Files         146      154       +8     
     Lines       24314    26195    +1881     
   ==========================================
   + Hits        18221    19889    +1668     
   - Misses       6093     6306     +213     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [datafusion/src/execution/context.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvZXhlY3V0aW9uL2NvbnRleHQucnM=) | `92.08% <ø> (+0.03%)` | :arrow_up: |
   | [datafusion/src/optimizer/simplify\_expressions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvb3B0aW1pemVyL3NpbXBsaWZ5X2V4cHJlc3Npb25zLnJz) | `87.45% <87.45%> (ø)` | |
   | [datafusion-cli/src/print\_format.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi1jbGkvc3JjL3ByaW50X2Zvcm1hdC5ycw==) | `81.25% <0.00%> (-9.17%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/mod.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9tb2QucnM=) | `78.70% <0.00%> (-4.06%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/window\_functions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi93aW5kb3dfZnVuY3Rpb25zLnJz) | `85.71% <0.00%> (-3.01%)` | :arrow_down: |
   | [...tafusion/src/physical\_plan/datetime\_expressions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9kYXRldGltZV9leHByZXNzaW9ucy5ycw==) | `67.29% <0.00%> (-2.52%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/common.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9jb21tb24ucnM=) | `84.21% <0.00%> (-2.00%)` | :arrow_down: |
   | [datafusion/src/optimizer/hash\_build\_probe\_order.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvb3B0aW1pemVyL2hhc2hfYnVpbGRfcHJvYmVfb3JkZXIucnM=) | `58.53% <0.00%> (-1.97%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/repartition.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9yZXBhcnRpdGlvbi5ycw==) | `82.45% <0.00%> (-1.89%)` | :arrow_down: |
   | [ballista/rust/scheduler/src/planner.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9zY2hlZHVsZXIvc3JjL3BsYW5uZXIucnM=) | `66.91% <0.00%> (-0.74%)` | :arrow_down: |
   | ... and [57 more](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [db4f098...af2c6ac](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643692661



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),

Review comment:
       This isn't correct if both sides can be null.




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r642675738



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false
+    }
+}
+
+fn is_zero<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(0))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 0. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 0. => true,
+        _ => false
+    }
+}
+
+fn is_true<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(left) => Expr::Negative(Box::new(simplify(right))),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if left == right => lit(0),

Review comment:
       I removed those by now from this PR https://github.com/apache/arrow-datafusion/pull/441 as I think they might be wrong in precense of nulls (`null - null` is not `0` ?) `null * 0` is not `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.

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r641941522



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,310 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::Expr;
+use crate::logical_plan::LogicalPlan;
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr { left, op: _, right } if left == right => simplify(left),

Review comment:
       You're right, I was wrongly assuming that all expressions are boolean so ... my mistake.
   I have updated. I hope I can check your other comments soon. Thanks !




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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643455939



##########
File path: datafusion/src/optimizer/mod.rs
##########
@@ -25,4 +25,5 @@ pub mod hash_build_probe_order;
 pub mod limit_push_down;
 pub mod optimizer;
 pub mod projection_push_down;
+pub mod remove_duplicate_filters;

Review comment:
       Ok, changed to simplify_expressions.




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

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



[GitHub] [arrow-datafusion] alamb commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

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


   Thanks again @jgoday 👍 


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643439834



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false
+    }
+}
+
+fn is_zero<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(0))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 0. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 0. => true,
+        _ => false
+    }
+}
+
+fn is_true<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(left) => Expr::Negative(Box::new(simplify(right))),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if left == right => lit(0),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_zero(left) || is_zero(right) => lit(0),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right
+        } if left == right => lit(1),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Plus,
+            right
+        } if is_zero(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Plus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => x.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *right.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *right.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *left.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *right.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => x.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *left.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => x.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    match plan {
+        LogicalPlan::Filter { input, predicate } => Ok(LogicalPlan::Filter {
+            input: input.clone(),
+            predicate: simplify(predicate),
+        }),
+        LogicalPlan::Projection { expr, input, schema } => Ok(LogicalPlan::Projection {
+            expr: expr.into_iter().map(|x| simplify(x)).collect::<Vec<_>>(),
+            input: input.clone(),
+            schema: schema.clone(),
+        }),
+        _ => {
+            let new_inputs = plan
+                .inputs()
+                .iter()
+                .map(|input| optimize(input))
+                .collect::<Result<Vec<_>>>()?;
+
+            let expr = plan.expressions();

Review comment:
       That looks right 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.

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643691688



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => x.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *right.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => *right.clone(),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => *left.clone(),
+                _ => expr.clone(),
+            })
+            .unwrap_or(expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)

Review comment:
       Also here - is this still needed and doesn't this remove certain oppurtunities as the recursive call isn't done?




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643691131



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),

Review comment:
       nvm this is actually correct 👍 




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

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



[GitHub] [arrow-datafusion] jgoday commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-854062163


   Thank you @Dandandan and @alamb  for all your help!
   
   


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

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



[GitHub] [arrow-datafusion] alamb commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r644357298



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule

Review comment:
       ```suggestion
   //! Simplify expressions optimizer rule
   ```

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_null(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(v) => v.is_null(),
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => !(*v),
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right && is_null(left) => *left.clone(),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&x.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*right.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*left.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*left.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    let new_inputs = plan
+        .inputs()
+        .iter()
+        .map(|input| optimize(input))
+        .collect::<Result<Vec<_>>>()?;
+    let expr = plan
+        .expressions()
+        .into_iter()
+        .map(|x| simplify(&x))
+        .collect::<Vec<_>>();
+    utils::from_plan(&plan, &expr, &new_inputs)
+}
+
+impl OptimizerRule for SimplifyExpressions {
+    fn name(&self) -> &str {
+        "simplify_expressions"
+    }
+
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        execution_props: &ExecutionProps,
+    ) -> Result<LogicalPlan> {
+        match plan {
+            LogicalPlan::Explain {
+                verbose,
+                plan,
+                stringified_plans,
+                schema,
+            } => {
+                let schema = schema.as_ref().to_owned().into();
+                optimize_explain(
+                    self,
+                    *verbose,
+                    &*plan,
+                    stringified_plans,
+                    &schema,
+                    execution_props,
+                )
+            }
+            _ => optimize(plan),
+        }
+    }
+}
+
+impl SimplifyExpressions {
+    #[allow(missing_docs)]
+    pub fn new() -> Self {
+        Self {}
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::logical_plan::{and, binary_expr, col, lit, Expr, LogicalPlanBuilder};
+    use crate::test::*;
+
+    fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) {
+        let rule = SimplifyExpressions::new();
+        let optimized_plan = rule
+            .optimize(plan, &ExecutionProps::new())
+            .expect("failed to optimize plan");
+        let formatted_plan = format!("{:?}", optimized_plan);
+        assert_eq!(formatted_plan, expected);
+    }
+
+    #[test]
+    fn test_simplify_or_true() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Or, lit(true));

Review comment:
       FWIW you can also create `or` operators using `Expr::or`
   
   For example:
   
   ```suggestion
           let expr_a = col("c").or(lit(true)));
   ```

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}

Review comment:
       I can't help but notice the similarity between this pass and `ConstantFolding`: https://github.com/apache/arrow-datafusion/blob/master/datafusion/src/optimizer/constant_folding.rs#L130
   
   Perhaps as a follow on PR, we could combine the code into a single pass

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_null(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(v) => v.is_null(),
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => !(*v),
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right && is_null(left) => *left.clone(),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&x.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*right.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*left.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*left.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    let new_inputs = plan
+        .inputs()
+        .iter()
+        .map(|input| optimize(input))
+        .collect::<Result<Vec<_>>>()?;
+    let expr = plan
+        .expressions()
+        .into_iter()
+        .map(|x| simplify(&x))
+        .collect::<Vec<_>>();
+    utils::from_plan(&plan, &expr, &new_inputs)
+}
+
+impl OptimizerRule for SimplifyExpressions {
+    fn name(&self) -> &str {
+        "simplify_expressions"
+    }
+
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        execution_props: &ExecutionProps,
+    ) -> Result<LogicalPlan> {
+        match plan {
+            LogicalPlan::Explain {
+                verbose,
+                plan,
+                stringified_plans,
+                schema,
+            } => {
+                let schema = schema.as_ref().to_owned().into();
+                optimize_explain(
+                    self,
+                    *verbose,
+                    &*plan,
+                    stringified_plans,
+                    &schema,
+                    execution_props,
+                )
+            }
+            _ => optimize(plan),
+        }
+    }
+}
+
+impl SimplifyExpressions {
+    #[allow(missing_docs)]
+    pub fn new() -> Self {
+        Self {}
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::logical_plan::{and, binary_expr, col, lit, Expr, LogicalPlanBuilder};
+    use crate::test::*;
+
+    fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) {
+        let rule = SimplifyExpressions::new();
+        let optimized_plan = rule
+            .optimize(plan, &ExecutionProps::new())
+            .expect("failed to optimize plan");
+        let formatted_plan = format!("{:?}", optimized_plan);
+        assert_eq!(formatted_plan, expected);
+    }
+
+    #[test]
+    fn test_simplify_or_true() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Or, lit(true));
+        let expr_b = binary_expr(lit(true), Operator::Or, col("c"));
+        let expected = lit(true);
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_false() -> Result<()> {
+        let expr_a = binary_expr(lit(false), Operator::Or, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::Or, lit(false));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Or, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_false() -> Result<()> {
+        let expr_a = binary_expr(lit(false), Operator::And, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::And, lit(false));
+        let expected = lit(false);
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::And, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_true() -> Result<()> {
+        let expr_a = binary_expr(lit(true), Operator::And, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::And, lit(true));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_multiply_by_one() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Multiply, lit(1));
+        let expr_b = binary_expr(lit(1), Operator::Multiply, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_divide_by_one() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Divide, lit(1));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_divide_by_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Divide, col("c"));
+        let expected = lit(1);
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_simple_and() -> Result<()> {
+        // (c > 5) AND (c > 5)
+        let expr = binary_expr(col("c").gt(lit(5)), Operator::And, col("c").gt(lit(5)));
+        let expected = col("c").gt(lit(5));
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_composed_and() -> Result<()> {
+        // ((c > 5) AND (d < 6)) AND (c > 5)
+        let expr = binary_expr(
+            binary_expr(col("c").gt(lit(5)), Operator::And, col("d").lt(lit(6))),
+            Operator::And,
+            col("c").gt(lit(5)),
+        );
+        let expected =
+            binary_expr(col("c").gt(lit(5)), Operator::And, col("d").lt(lit(6)));
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_negated_and() -> Result<()> {
+        // (c > 5) AND !(c > 5) -- can't remove
+        let expr = binary_expr(
+            col("c").gt(lit(5)),
+            Operator::And,
+            Expr::not(col("c").gt(lit(5))),
+        );
+        let expected = expr.clone();
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_and() -> Result<()> {
+        // (c > 5) OR ((d < 6) AND (c > 5) -- can't remove

Review comment:
       ```suggestion
           // (c > 5) OR ((d < 6) AND (c > 5) -- can remove
   ```

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file

Review comment:
       ```suggestion
   // Licensed to the Apache Software Foundation (ASF) under one
   // or more contributor license agreements.  See the NOTICE file
   // distributed with this work for additional information
   // regarding copyright ownership.  The ASF licenses this file
   ```
   
   The first few lines seem to be missing from the RAT

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_null(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(v) => v.is_null(),
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => !(*v),
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right && is_null(left) => *left.clone(),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&x.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*right.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*left.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*left.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    let new_inputs = plan
+        .inputs()
+        .iter()
+        .map(|input| optimize(input))
+        .collect::<Result<Vec<_>>>()?;
+    let expr = plan
+        .expressions()
+        .into_iter()
+        .map(|x| simplify(&x))
+        .collect::<Vec<_>>();
+    utils::from_plan(&plan, &expr, &new_inputs)
+}
+
+impl OptimizerRule for SimplifyExpressions {
+    fn name(&self) -> &str {
+        "simplify_expressions"
+    }
+
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        execution_props: &ExecutionProps,
+    ) -> Result<LogicalPlan> {
+        match plan {
+            LogicalPlan::Explain {
+                verbose,
+                plan,
+                stringified_plans,
+                schema,
+            } => {
+                let schema = schema.as_ref().to_owned().into();
+                optimize_explain(
+                    self,
+                    *verbose,
+                    &*plan,
+                    stringified_plans,
+                    &schema,
+                    execution_props,
+                )
+            }
+            _ => optimize(plan),
+        }
+    }
+}
+
+impl SimplifyExpressions {
+    #[allow(missing_docs)]
+    pub fn new() -> Self {
+        Self {}
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::logical_plan::{and, binary_expr, col, lit, Expr, LogicalPlanBuilder};
+    use crate::test::*;
+
+    fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) {
+        let rule = SimplifyExpressions::new();
+        let optimized_plan = rule
+            .optimize(plan, &ExecutionProps::new())
+            .expect("failed to optimize plan");
+        let formatted_plan = format!("{:?}", optimized_plan);
+        assert_eq!(formatted_plan, expected);
+    }
+
+    #[test]
+    fn test_simplify_or_true() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Or, lit(true));
+        let expr_b = binary_expr(lit(true), Operator::Or, col("c"));
+        let expected = lit(true);
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_false() -> Result<()> {
+        let expr_a = binary_expr(lit(false), Operator::Or, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::Or, lit(false));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Or, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_false() -> Result<()> {
+        let expr_a = binary_expr(lit(false), Operator::And, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::And, lit(false));
+        let expected = lit(false);
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::And, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_true() -> Result<()> {
+        let expr_a = binary_expr(lit(true), Operator::And, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::And, lit(true));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_multiply_by_one() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Multiply, lit(1));
+        let expr_b = binary_expr(lit(1), Operator::Multiply, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_divide_by_one() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Divide, lit(1));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_divide_by_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Divide, col("c"));

Review comment:
       I wonder if we care that in some cases (where c has a `0`) this simplification will avoid a runtime error when it would have generated one without the optimization pass
   
   I personally think it is ok, but wanted to mention 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.

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



[GitHub] [arrow-datafusion] alamb commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

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


   Thanks again @jgoday 👍 


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r642666851



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {

Review comment:
       These are really good use cases for the `matches!`  macro https://doc.rust-lang.org/std/macro.matches.html




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

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



[GitHub] [arrow-datafusion] Dandandan commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-853977793


   Awesome thanks @jgoday 💯 


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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643818548



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)

Review comment:
       I think that is needed for expressions like '((c > 5) AND (d < 6)) AND (c > 5)'.
   Nested simplification should be fixed by latest commit.




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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643818858



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),

Review comment:
       You're right, should be fixed now.
   




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

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



[GitHub] [arrow-datafusion] codecov-commenter commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-852434776


   # [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#436](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (25adf53) into [master](https://codecov.io/gh/apache/arrow-datafusion/commit/db4f098d38993b96ce1134c4bc7bf5c6579509cf?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (db4f098) will **increase** coverage by `1.00%`.
   > The diff coverage is `86.61%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-datafusion/pull/436/graphs/tree.svg?width=650&height=150&src=pr&token=JXwWBKD3D9&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #436      +/-   ##
   ==========================================
   + Coverage   74.94%   75.94%   +1.00%     
   ==========================================
     Files         146      154       +8     
     Lines       24314    26126    +1812     
   ==========================================
   + Hits        18221    19842    +1621     
   - Misses       6093     6284     +191     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [datafusion/src/execution/context.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvZXhlY3V0aW9uL2NvbnRleHQucnM=) | `92.08% <ø> (+0.03%)` | :arrow_up: |
   | [datafusion/src/optimizer/simplify\_expressions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvb3B0aW1pemVyL3NpbXBsaWZ5X2V4cHJlc3Npb25zLnJz) | `86.61% <86.61%> (ø)` | |
   | [datafusion-cli/src/print\_format.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi1jbGkvc3JjL3ByaW50X2Zvcm1hdC5ycw==) | `81.25% <0.00%> (-9.17%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/mod.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9tb2QucnM=) | `78.70% <0.00%> (-4.06%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/window\_functions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi93aW5kb3dfZnVuY3Rpb25zLnJz) | `85.71% <0.00%> (-3.01%)` | :arrow_down: |
   | [...tafusion/src/physical\_plan/datetime\_expressions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9kYXRldGltZV9leHByZXNzaW9ucy5ycw==) | `67.29% <0.00%> (-2.52%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/common.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9jb21tb24ucnM=) | `84.21% <0.00%> (-2.00%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/repartition.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9yZXBhcnRpdGlvbi5ycw==) | `82.45% <0.00%> (-1.89%)` | :arrow_down: |
   | [ballista/rust/scheduler/src/planner.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9zY2hlZHVsZXIvc3JjL3BsYW5uZXIucnM=) | `66.91% <0.00%> (-0.74%)` | :arrow_down: |
   | [datafusion/tests/sql.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi90ZXN0cy9zcWwucnM=) | `99.27% <0.00%> (-0.62%)` | :arrow_down: |
   | ... and [49 more](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [db4f098...25adf53](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

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



[GitHub] [arrow-datafusion] jgoday commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-851664134


   @Dandandan I've just added some more simplification rules (from https://github.com/Dandandan/datafusion-tokomak/blob/main/src/lib.rs#L44, as you mentioned before), What do you think ?
   
   Still have to test and add more LogicalPlan cases (Projection, Window, Aggregation, ...) to optimize function.


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

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



[GitHub] [arrow-datafusion] jgoday commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-852951404


   > 
   > 
   > It looks good to me @jgoday
   > Could you fix the clippy and formatting errors?
   
   Should be fixed now 👍 


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643689897



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),

Review comment:
       This is also not correct I think if a `null` is present




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r642675738



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false
+    }
+}
+
+fn is_zero<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(0))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 0. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 0. => true,
+        _ => false
+    }
+}
+
+fn is_true<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(left) => Expr::Negative(Box::new(simplify(right))),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if left == right => lit(0),

Review comment:
       I removed those by now from this PR https://github.com/apache/arrow-datafusion/pull/441 as I think they might be wrong in precense of nulls (null - null` is not `0` ?) `null * 0` is not `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.

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



[GitHub] [arrow-datafusion] codecov-commenter edited a comment on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-852434776


   # [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#436](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c79899b) into [master](https://codecov.io/gh/apache/arrow-datafusion/commit/db4f098d38993b96ce1134c4bc7bf5c6579509cf?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (db4f098) will **increase** coverage by `1.02%`.
   > The diff coverage is `87.45%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-datafusion/pull/436/graphs/tree.svg?width=650&height=150&src=pr&token=JXwWBKD3D9&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #436      +/-   ##
   ==========================================
   + Coverage   74.94%   75.96%   +1.02%     
   ==========================================
     Files         146      154       +8     
     Lines       24314    26147    +1833     
   ==========================================
   + Hits        18221    19863    +1642     
   - Misses       6093     6284     +191     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [datafusion/src/execution/context.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvZXhlY3V0aW9uL2NvbnRleHQucnM=) | `92.08% <ø> (+0.03%)` | :arrow_up: |
   | [datafusion/src/optimizer/simplify\_expressions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvb3B0aW1pemVyL3NpbXBsaWZ5X2V4cHJlc3Npb25zLnJz) | `87.45% <87.45%> (ø)` | |
   | [datafusion-cli/src/print\_format.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi1jbGkvc3JjL3ByaW50X2Zvcm1hdC5ycw==) | `81.25% <0.00%> (-9.17%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/mod.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9tb2QucnM=) | `78.70% <0.00%> (-4.06%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/window\_functions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi93aW5kb3dfZnVuY3Rpb25zLnJz) | `85.71% <0.00%> (-3.01%)` | :arrow_down: |
   | [...tafusion/src/physical\_plan/datetime\_expressions.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9kYXRldGltZV9leHByZXNzaW9ucy5ycw==) | `67.29% <0.00%> (-2.52%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/common.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9jb21tb24ucnM=) | `84.21% <0.00%> (-2.00%)` | :arrow_down: |
   | [datafusion/src/physical\_plan/repartition.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi9zcmMvcGh5c2ljYWxfcGxhbi9yZXBhcnRpdGlvbi5ycw==) | `82.45% <0.00%> (-1.89%)` | :arrow_down: |
   | [ballista/rust/scheduler/src/planner.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9zY2hlZHVsZXIvc3JjL3BsYW5uZXIucnM=) | `66.91% <0.00%> (-0.74%)` | :arrow_down: |
   | [datafusion/tests/sql.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGF0YWZ1c2lvbi90ZXN0cy9zcWwucnM=) | `99.27% <0.00%> (-0.62%)` | :arrow_down: |
   | ... and [49 more](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [db4f098...c79899b](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643440084



##########
File path: datafusion/src/optimizer/mod.rs
##########
@@ -25,4 +25,5 @@ pub mod hash_build_probe_order;
 pub mod limit_push_down;
 pub mod optimizer;
 pub mod projection_push_down;
+pub mod remove_duplicate_filters;

Review comment:
       I think that would be a more accurate name by now 👍 




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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r642666699



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {

Review comment:
       any reason to use lifetime specifiers everywhere - we can't use just use `&Expr` in 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.

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



[GitHub] [arrow-datafusion] Dandandan commented on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-850792952


   Looks like a good start.
   
   I think it's good to think of these rules as simple rewrite rules we can apply to the expression tree.
   
   ```
   p OR p => a
   p AND p => a
   ```
   
   Some weeks ago I implemented some rules in this custom DataFusion optimizer (based on graph rewriting), you can see them here:
   
   https://github.com/Dandandan/datafusion-tokomak/blob/main/src/lib.rs#L44
   There are a lot of other patterns we can add if we do term rewriting in this way.
   
   Also, I think we should make this optimizer run on *any* `Expr`, not just those in `Filter` as this would benefit any `Expr` (in projections, aggregations, joins, etc)
   
   


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r644922975



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,506 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),

Review comment:
       I merged it, but see this part might still be not 100% correct.
   
   if left and right are a column, the value can still be null (for some evaluations).
   Also the division by 0 (e.g. 0/0 or x/x where x is 0) is tricky.
   I think there is no easy way to do it safely.




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

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



[GitHub] [arrow-datafusion] alamb commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r644357298



##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule

Review comment:
       ```suggestion
   //! Simplify expressions optimizer rule
   ```

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_null(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(v) => v.is_null(),
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => !(*v),
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right && is_null(left) => *left.clone(),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&x.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*right.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*left.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*left.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    let new_inputs = plan
+        .inputs()
+        .iter()
+        .map(|input| optimize(input))
+        .collect::<Result<Vec<_>>>()?;
+    let expr = plan
+        .expressions()
+        .into_iter()
+        .map(|x| simplify(&x))
+        .collect::<Vec<_>>();
+    utils::from_plan(&plan, &expr, &new_inputs)
+}
+
+impl OptimizerRule for SimplifyExpressions {
+    fn name(&self) -> &str {
+        "simplify_expressions"
+    }
+
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        execution_props: &ExecutionProps,
+    ) -> Result<LogicalPlan> {
+        match plan {
+            LogicalPlan::Explain {
+                verbose,
+                plan,
+                stringified_plans,
+                schema,
+            } => {
+                let schema = schema.as_ref().to_owned().into();
+                optimize_explain(
+                    self,
+                    *verbose,
+                    &*plan,
+                    stringified_plans,
+                    &schema,
+                    execution_props,
+                )
+            }
+            _ => optimize(plan),
+        }
+    }
+}
+
+impl SimplifyExpressions {
+    #[allow(missing_docs)]
+    pub fn new() -> Self {
+        Self {}
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::logical_plan::{and, binary_expr, col, lit, Expr, LogicalPlanBuilder};
+    use crate::test::*;
+
+    fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) {
+        let rule = SimplifyExpressions::new();
+        let optimized_plan = rule
+            .optimize(plan, &ExecutionProps::new())
+            .expect("failed to optimize plan");
+        let formatted_plan = format!("{:?}", optimized_plan);
+        assert_eq!(formatted_plan, expected);
+    }
+
+    #[test]
+    fn test_simplify_or_true() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Or, lit(true));

Review comment:
       FWIW you can also create `or` operators using `Expr::or`
   
   For example:
   
   ```suggestion
           let expr_a = col("c").or(lit(true)));
   ```

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}

Review comment:
       I can't help but notice the similarity between this pass and `ConstantFolding`: https://github.com/apache/arrow-datafusion/blob/master/datafusion/src/optimizer/constant_folding.rs#L130
   
   Perhaps as a follow on PR, we could combine the code into a single pass

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_null(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(v) => v.is_null(),
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => !(*v),
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right && is_null(left) => *left.clone(),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&x.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*right.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*left.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*left.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    let new_inputs = plan
+        .inputs()
+        .iter()
+        .map(|input| optimize(input))
+        .collect::<Result<Vec<_>>>()?;
+    let expr = plan
+        .expressions()
+        .into_iter()
+        .map(|x| simplify(&x))
+        .collect::<Vec<_>>();
+    utils::from_plan(&plan, &expr, &new_inputs)
+}
+
+impl OptimizerRule for SimplifyExpressions {
+    fn name(&self) -> &str {
+        "simplify_expressions"
+    }
+
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        execution_props: &ExecutionProps,
+    ) -> Result<LogicalPlan> {
+        match plan {
+            LogicalPlan::Explain {
+                verbose,
+                plan,
+                stringified_plans,
+                schema,
+            } => {
+                let schema = schema.as_ref().to_owned().into();
+                optimize_explain(
+                    self,
+                    *verbose,
+                    &*plan,
+                    stringified_plans,
+                    &schema,
+                    execution_props,
+                )
+            }
+            _ => optimize(plan),
+        }
+    }
+}
+
+impl SimplifyExpressions {
+    #[allow(missing_docs)]
+    pub fn new() -> Self {
+        Self {}
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::logical_plan::{and, binary_expr, col, lit, Expr, LogicalPlanBuilder};
+    use crate::test::*;
+
+    fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) {
+        let rule = SimplifyExpressions::new();
+        let optimized_plan = rule
+            .optimize(plan, &ExecutionProps::new())
+            .expect("failed to optimize plan");
+        let formatted_plan = format!("{:?}", optimized_plan);
+        assert_eq!(formatted_plan, expected);
+    }
+
+    #[test]
+    fn test_simplify_or_true() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Or, lit(true));
+        let expr_b = binary_expr(lit(true), Operator::Or, col("c"));
+        let expected = lit(true);
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_false() -> Result<()> {
+        let expr_a = binary_expr(lit(false), Operator::Or, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::Or, lit(false));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Or, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_false() -> Result<()> {
+        let expr_a = binary_expr(lit(false), Operator::And, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::And, lit(false));
+        let expected = lit(false);
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::And, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_true() -> Result<()> {
+        let expr_a = binary_expr(lit(true), Operator::And, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::And, lit(true));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_multiply_by_one() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Multiply, lit(1));
+        let expr_b = binary_expr(lit(1), Operator::Multiply, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_divide_by_one() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Divide, lit(1));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_divide_by_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Divide, col("c"));
+        let expected = lit(1);
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_simple_and() -> Result<()> {
+        // (c > 5) AND (c > 5)
+        let expr = binary_expr(col("c").gt(lit(5)), Operator::And, col("c").gt(lit(5)));
+        let expected = col("c").gt(lit(5));
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_composed_and() -> Result<()> {
+        // ((c > 5) AND (d < 6)) AND (c > 5)
+        let expr = binary_expr(
+            binary_expr(col("c").gt(lit(5)), Operator::And, col("d").lt(lit(6))),
+            Operator::And,
+            col("c").gt(lit(5)),
+        );
+        let expected =
+            binary_expr(col("c").gt(lit(5)), Operator::And, col("d").lt(lit(6)));
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_negated_and() -> Result<()> {
+        // (c > 5) AND !(c > 5) -- can't remove
+        let expr = binary_expr(
+            col("c").gt(lit(5)),
+            Operator::And,
+            Expr::not(col("c").gt(lit(5))),
+        );
+        let expected = expr.clone();
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_and() -> Result<()> {
+        // (c > 5) OR ((d < 6) AND (c > 5) -- can't remove

Review comment:
       ```suggestion
           // (c > 5) OR ((d < 6) AND (c > 5) -- can remove
   ```

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file

Review comment:
       ```suggestion
   // Licensed to the Apache Software Foundation (ASF) under one
   // or more contributor license agreements.  See the NOTICE file
   // distributed with this work for additional information
   // regarding copyright ownership.  The ASF licenses this file
   ```
   
   The first few lines seem to be missing from the RAT

##########
File path: datafusion/src/optimizer/simplify_expressions.rs
##########
@@ -0,0 +1,538 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Simplify expressions optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: b > 2 AND b > 2
+/// is optimized to
+/// Filter: b > 2
+pub struct SimplifyExpressions {}
+
+fn expr_contains(expr: &Expr, needle: &Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr(expr: &Expr) -> Option<&Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: Operator) -> bool {
+    op == Operator::And || op == Operator::Or
+}
+
+fn is_one(s: &Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1)))
+        | Expr::Literal(ScalarValue::Int16(Some(1)))
+        | Expr::Literal(ScalarValue::Int32(Some(1)))
+        | Expr::Literal(ScalarValue::Int64(Some(1)))
+        | Expr::Literal(ScalarValue::UInt8(Some(1)))
+        | Expr::Literal(ScalarValue::UInt16(Some(1)))
+        | Expr::Literal(ScalarValue::UInt32(Some(1)))
+        | Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false,
+    }
+}
+
+fn is_true(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_null(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(v) => v.is_null(),
+        _ => false,
+    }
+}
+
+fn is_false(expr: &Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => !(*v),
+        _ => false,
+    }
+}
+
+fn simplify(expr: &Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Multiply,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if is_one(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right && is_null(left) => *left.clone(),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Divide,
+            right,
+        } if left == right => lit(1),
+        Expr::BinaryExpr { left, op, right }
+            if left == right && operator_is_boolean(*op) =>
+        {
+            simplify(left)
+        }
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&x.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*right.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&*left.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(left, right) => as_binary_expr(left)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*right.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if expr_contains(right, left) => as_binary_expr(right)
+            .map(|x| match x {
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::Or,
+                    right: _,
+                } => simplify(&*left.clone()),
+                Expr::BinaryExpr {
+                    left: _,
+                    op: Operator::And,
+                    right: _,
+                } => simplify(&x.clone()),
+                _ => expr.clone(),
+            })
+            .unwrap_or_else(|| expr.clone()),
+        Expr::BinaryExpr { left, op, right } => Expr::BinaryExpr {
+            left: Box::new(simplify(&left)),
+            op: *op,
+            right: Box::new(simplify(right)),
+        },
+        _ => expr.clone(),
+    }
+}
+
+fn optimize(plan: &LogicalPlan) -> Result<LogicalPlan> {
+    let new_inputs = plan
+        .inputs()
+        .iter()
+        .map(|input| optimize(input))
+        .collect::<Result<Vec<_>>>()?;
+    let expr = plan
+        .expressions()
+        .into_iter()
+        .map(|x| simplify(&x))
+        .collect::<Vec<_>>();
+    utils::from_plan(&plan, &expr, &new_inputs)
+}
+
+impl OptimizerRule for SimplifyExpressions {
+    fn name(&self) -> &str {
+        "simplify_expressions"
+    }
+
+    fn optimize(
+        &self,
+        plan: &LogicalPlan,
+        execution_props: &ExecutionProps,
+    ) -> Result<LogicalPlan> {
+        match plan {
+            LogicalPlan::Explain {
+                verbose,
+                plan,
+                stringified_plans,
+                schema,
+            } => {
+                let schema = schema.as_ref().to_owned().into();
+                optimize_explain(
+                    self,
+                    *verbose,
+                    &*plan,
+                    stringified_plans,
+                    &schema,
+                    execution_props,
+                )
+            }
+            _ => optimize(plan),
+        }
+    }
+}
+
+impl SimplifyExpressions {
+    #[allow(missing_docs)]
+    pub fn new() -> Self {
+        Self {}
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::logical_plan::{and, binary_expr, col, lit, Expr, LogicalPlanBuilder};
+    use crate::test::*;
+
+    fn assert_optimized_plan_eq(plan: &LogicalPlan, expected: &str) {
+        let rule = SimplifyExpressions::new();
+        let optimized_plan = rule
+            .optimize(plan, &ExecutionProps::new())
+            .expect("failed to optimize plan");
+        let formatted_plan = format!("{:?}", optimized_plan);
+        assert_eq!(formatted_plan, expected);
+    }
+
+    #[test]
+    fn test_simplify_or_true() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Or, lit(true));
+        let expr_b = binary_expr(lit(true), Operator::Or, col("c"));
+        let expected = lit(true);
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_false() -> Result<()> {
+        let expr_a = binary_expr(lit(false), Operator::Or, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::Or, lit(false));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_or_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Or, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_false() -> Result<()> {
+        let expr_a = binary_expr(lit(false), Operator::And, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::And, lit(false));
+        let expected = lit(false);
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::And, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_and_true() -> Result<()> {
+        let expr_a = binary_expr(lit(true), Operator::And, col("c"));
+        let expr_b = binary_expr(col("c"), Operator::And, lit(true));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_multiply_by_one() -> Result<()> {
+        let expr_a = binary_expr(col("c"), Operator::Multiply, lit(1));
+        let expr_b = binary_expr(lit(1), Operator::Multiply, col("c"));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr_a), expected);
+        assert_eq!(simplify(&expr_b), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_divide_by_one() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Divide, lit(1));
+        let expected = col("c");
+
+        assert_eq!(simplify(&expr), expected);
+        Ok(())
+    }
+
+    #[test]
+    fn test_simplify_divide_by_same() -> Result<()> {
+        let expr = binary_expr(col("c"), Operator::Divide, col("c"));

Review comment:
       I wonder if we care that in some cases (where c has a `0`) this simplification will avoid a runtime error when it would have generated one without the optimization pass
   
   I personally think it is ok, but wanted to mention 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.

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r641908160



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,310 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::Expr;
+use crate::logical_plan::LogicalPlan;
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr { left, op: _, right } if left == right => simplify(left),

Review comment:
       I am seeing one other example from the test `1=1` is also a good example, it now is rewritten to `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.

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643405664



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false
+    }
+}
+
+fn is_zero<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(0))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 0. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 0. => true,
+        _ => false
+    }
+}
+
+fn is_true<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(left) => Expr::Negative(Box::new(simplify(right))),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if left == right => lit(0),

Review comment:
       I agree, already removed.
   maybe there should be some specific optimizations for arithmetic operators ...




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

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



[GitHub] [arrow-datafusion] codecov-commenter edited a comment on pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#issuecomment-852434776


   # [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#436](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e3fd07e) into [master](https://codecov.io/gh/apache/arrow-datafusion/commit/db4f098d38993b96ce1134c4bc7bf5c6579509cf?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (db4f098) will **increase** coverage by `1.02%`.
   > The diff coverage is `75.73%`.
   
   > :exclamation: Current head e3fd07e differs from pull request most recent head 2e777a8. Consider uploading reports for the commit 2e777a8 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow-datafusion/pull/436/graphs/tree.svg?width=650&height=150&src=pr&token=JXwWBKD3D9&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master     #436      +/-   ##
   ==========================================
   + Coverage   74.94%   75.96%   +1.02%     
   ==========================================
     Files         146      154       +8     
     Lines       24314    26148    +1834     
   ==========================================
   + Hits        18221    19864    +1643     
   - Misses       6093     6284     +191     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [ballista/rust/client/src/columnar\_batch.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jbGllbnQvc3JjL2NvbHVtbmFyX2JhdGNoLnJz) | `0.00% <ø> (ø)` | |
   | [ballista/rust/client/src/context.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jbGllbnQvc3JjL2NvbnRleHQucnM=) | `0.00% <0.00%> (ø)` | |
   | [ballista/rust/core/src/client.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jb3JlL3NyYy9jbGllbnQucnM=) | `0.00% <ø> (ø)` | |
   | [ballista/rust/core/src/datasource.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jb3JlL3NyYy9kYXRhc291cmNlLnJz) | `0.00% <ø> (ø)` | |
   | [ballista/rust/core/src/error.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jb3JlL3NyYy9lcnJvci5ycw==) | `0.00% <ø> (ø)` | |
   | [...ta/rust/core/src/execution\_plans/shuffle\_reader.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jb3JlL3NyYy9leGVjdXRpb25fcGxhbnMvc2h1ZmZsZV9yZWFkZXIucnM=) | `0.00% <0.00%> (ø)` | |
   | [...ust/core/src/execution\_plans/unresolved\_shuffle.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jb3JlL3NyYy9leGVjdXRpb25fcGxhbnMvdW5yZXNvbHZlZF9zaHVmZmxlLnJz) | `50.00% <ø> (ø)` | |
   | [ballista/rust/core/src/memory\_stream.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jb3JlL3NyYy9tZW1vcnlfc3RyZWFtLnJz) | `60.00% <ø> (+60.00%)` | :arrow_up: |
   | [...ta/rust/core/src/serde/physical\_plan/from\_proto.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jb3JlL3NyYy9zZXJkZS9waHlzaWNhbF9wbGFuL2Zyb21fcHJvdG8ucnM=) | `39.78% <ø> (ø)` | |
   | [ballista/rust/core/src/serde/physical\_plan/mod.rs](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-YmFsbGlzdGEvcnVzdC9jb3JlL3NyYy9zZXJkZS9waHlzaWNhbF9wbGFuL21vZC5ycw==) | `100.00% <ø> (ø)` | |
   | ... and [76 more](https://codecov.io/gh/apache/arrow-datafusion/pull/436/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [db4f098...2e777a8](https://codecov.io/gh/apache/arrow-datafusion/pull/436?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

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



[GitHub] [arrow-datafusion] Dandandan commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
Dandandan commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643441081



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(1))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(1))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(1))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 1. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 1. => true,
+        _ => false
+    }
+}
+
+fn is_zero<'a>(s: &'a Expr) -> bool {
+    match s {
+        Expr::Literal(ScalarValue::Int8(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int16(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int32(Some(0))) => true,
+        Expr::Literal(ScalarValue::Int64(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt8(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt16(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt32(Some(0))) => true,
+        Expr::Literal(ScalarValue::UInt64(Some(0))) => true,
+        Expr::Literal(ScalarValue::Float32(Some(v))) if *v == 0. => true,
+        Expr::Literal(ScalarValue::Float64(Some(v))) if *v == 0. => true,
+        _ => false
+    }
+}
+
+fn is_true<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v,
+        _ => false,
+    }
+}
+
+fn is_false<'a>(expr: &'a Expr) -> bool {
+    match expr {
+        Expr::Literal(ScalarValue::Boolean(Some(v))) => *v == false,
+        _ => false,
+    }
+}
+
+fn simplify<'a>(expr: &'a Expr) -> Expr {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_true(left) || is_true(right) => lit(true),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if is_false(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } if left == right => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_false(left) || is_false(right) => lit(false),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if is_true(left) => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } if left == right => simplify(right),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(left) => Expr::Negative(Box::new(simplify(right))),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if is_zero(right) => simplify(left),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Minus,
+            right
+        } if left == right => lit(0),

Review comment:
       It might be possible to do the optimization if we know the input is non null based on the schema. But probably something to find out later 👍 




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

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



[GitHub] [arrow-datafusion] jgoday commented on a change in pull request #436: Remove reundant filters (e.g. c> 5 AND c>5 --> c>5)

Posted by GitBox <gi...@apache.org>.
jgoday commented on a change in pull request #436:
URL: https://github.com/apache/arrow-datafusion/pull/436#discussion_r643411259



##########
File path: datafusion/src/optimizer/remove_duplicate_filters.rs
##########
@@ -0,0 +1,611 @@
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+//! Remove duplicate filters optimizer rule
+
+use crate::execution::context::ExecutionProps;
+use crate::logical_plan::LogicalPlan;
+use crate::logical_plan::{lit, Expr};
+use crate::optimizer::optimizer::OptimizerRule;
+use crate::optimizer::utils;
+use crate::optimizer::utils::optimize_explain;
+use crate::scalar::ScalarValue;
+use crate::{error::Result, logical_plan::Operator};
+
+/// Remove duplicate filters optimizer.
+/// # Introduction
+/// It uses boolean algebra laws to simplify or reduce the number of terms in expressions.
+///
+/// Filter: #b Gt Int32(2) And #b Gt Int32(2)
+/// is optimized to
+/// Filter: #b Gt Int32(2)
+pub struct RemoveDuplicateFilters {}
+
+fn expr_contains<'a>(expr: &'a Expr, needle: &'a Expr) -> bool {
+    match expr {
+        Expr::BinaryExpr {
+            left,
+            op: Operator::And,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        Expr::BinaryExpr {
+            left,
+            op: Operator::Or,
+            right,
+        } => expr_contains(left, needle) || expr_contains(right, needle),
+        _ => expr == needle,
+    }
+}
+
+fn as_binary_expr<'a>(expr: &'a Expr) -> Option<&'a Expr> {
+    match expr {
+        Expr::BinaryExpr { .. } => Some(expr),
+        _ => None,
+    }
+}
+
+fn operator_is_boolean(op: &Operator) -> bool {
+    op == &Operator::And || op == &Operator::Or
+}
+
+fn is_one<'a>(s: &'a Expr) -> bool {
+    match s {

Review comment:
       Ok, nice macro (I did not know it).
   
   Sorry if i'm being too clumsy, but how can I use it if I have to guard two different cases (each one with a different variable/type) ?
   ```
   matches!(s,
      Scalar::Int8(1) |
      Scalar::Int16(1) |
      Scalar::Float32(Some(v)) if *v == 1. |
      Scalar::Float64(Some(v)) if *v == 1. )
   ```
   
   Seems that maches! only allows one last guard, and the existing binding must have the same type in all alternatives.
   
   Is there any other way ?
   
   
   




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

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