You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by GitBox <gi...@apache.org> on 2020/11/11 09:38:40 UTC

[GitHub] [calcite] rubenada commented on a change in pull request #2253: [CALCITE-4385] Simplifies AND/OR condition that has common expressions, extract and eliminate/merge them as much as possible (Jiatao Tao)

rubenada commented on a change in pull request #2253:
URL: https://github.com/apache/calcite/pull/2253#discussion_r521232496



##########
File path: core/src/main/java/org/apache/calcite/rex/RexSimplify.java
##########
@@ -1689,6 +1690,123 @@ RexNode simplifyAnd2ForUnknownAsFalse(List<RexNode> terms,
     }
   }
 
+  /**
+   * <p>Simplifies AND/OR condition that has common expressions,
+   * extract and eliminate/merge them as much as possible.</p>
+   * <ul>
+   * <li>(a OR b) AND (a OR b OR c OR d) =&gt; (a OR b)
+   * <li>(a OR b OR c OR d) AND (a OR b OR e OR f) =&gt; (a OR b) OR ((c OR d) AND (e OR f))
+
+   * <li>(a AND b) OR (a AND b AND c) =&gt; (a AND b)
+   * <li>(a AND b AND c AND d) OR (a AND b AND e AND f) =&gt;
+   * (a AND b) AND ((c AND d) OR (e AND f))
+   * </ul>
+   * <p> The difference between {@link #simplifyAnd} is that {@link #simplifyAnd} mainly
+   * simplifies expressions whose answer can be determined without evaluating both sides,
+   * like: FALSE AND (xxx) =&gt; FALSE</p>
+   */
+  public RexNode eliminateCommonExprInCondition(RexNode call) {
+    if (!(call instanceof RexCall)) {
+      return call;
+    }
+    // Simplify children recursively
+    // so that when we simplify parent node
+    // the child node should be simplified
+    List<RexNode> operands = ((RexCall) call).getOperands();
+    List<RexNode> simplifiedChildren = operands.stream()
+        .map(this::eliminateCommonExprInCondition)
+        .collect(Collectors.toList());
+
+    switch (call.getKind()) {
+    case AND: {
+      // Split the children to get the disjunctive predicates.
+      List<List<RexNode>> disjunctions = simplifiedChildren.stream()
+          .map(RelOptUtil::disjunctions)
+          .collect(Collectors.toList());
+
+      // Find the common predict among the list by retainAll.
+      List<RexNode> common = new ArrayList<>(disjunctions.get(0));
+      disjunctions.forEach(common::retainAll);
+
+      if (common.isEmpty()) {
+        // no common, return the original but with child simplified
+        return and(simplifiedChildren);
+      } else {
+        // Get the predicates that without the common part.
+        disjunctions.forEach(p -> p.removeAll(common));
+        if (disjunctions.stream().anyMatch(List::isEmpty)) {
+          // (a || b) && (a || b || c || d) => (a || b)
+          return or(common);
+        } else {
+          // (a || b || c || d) && (a || b || e || f) =>
+          // (a || b) || ((c || d) && (e || f))
+          List<RexNode> splice = disjunctions.stream()
+              .map(this::or)
+              .collect(Collectors.toList());
+          return or(or(common), and(splice));
+        }
+      }
+    }
+    case OR: {
+      // Split the children to get the conjunctions predicates.
+      List<List<RexNode>> conjunctions = simplifiedChildren.stream()
+          .map(RelOptUtil::conjunctions)
+          .collect(Collectors.toList());
+
+      // Find the common predict among the list by retainAll.
+      List<RexNode> common = new ArrayList<>(conjunctions.get(0));
+      conjunctions.forEach(common::retainAll);
+
+      if (common.isEmpty()) {
+        // no common, return the original but with child simplified
+        return or(simplifiedChildren);
+      } else {
+        // Get the predicates that without the common part.
+        conjunctions.forEach(p -> p.removeAll(common));
+        if (conjunctions.stream().anyMatch(List::isEmpty)) {
+          // (a && b) || (a && b && c) => (a && b)
+          return and(common);
+        } else {
+          // (a && b && c && d) || (a && b && e && f) =>
+          // (a && b) && ((c && d) || (e && f))
+          List<RexNode> splice = conjunctions.stream()
+              .map(this::and)
+              .collect(Collectors.toList());
+          return and(and(common), or(splice));
+        }
+      }
+    }
+    default:
+      return call;
+    }
+  }
+
+  private RexNode and(RexNode... nodes) {
+    return and(ImmutableList.copyOf(nodes));
+  }
+
+  private RexNode and(List<RexNode> nodes) {
+    if (nodes.size() == 1) {
+      return nodes.get(0);
+    } else {
+      return rexBuilder.makeCall(SqlStdOperatorTable.AND,
+          ImmutableList.copyOf(nodes));
+    }
+  }
+
+  private RexNode or(RexNode... nodes) {
+    return or(ImmutableList.copyOf(nodes));
+  }
+
+  private RexNode or(List<RexNode> nodes) {
+    if (nodes.size() == 1) {
+      return nodes.get(0);
+    } else {
+      return rexBuilder.makeCall(SqlStdOperatorTable.OR,
+          ImmutableList.copyOf(nodes));
+    }
+  }

Review comment:
       Maybe RexUtil.composeDisjunction / RexUtil.composeConjunction




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