You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2016/03/17 23:17:05 UTC

[1/2] calcite git commit: [CALCITE-1116] Extend simplify for reducing expressions [Forced Update!]

Repository: calcite
Updated Branches:
  refs/heads/master 82470e32d -> 0ad58ed62 (forced update)


[CALCITE-1116] Extend simplify for reducing expressions

Close apache/calcite#202


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/3a54e54f
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/3a54e54f
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/3a54e54f

Branch: refs/heads/master
Commit: 3a54e54fadfcf383693555ac753e88edb8f3f5d6
Parents: 94f8837
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Mar 1 13:28:19 2016 +0100
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Mar 17 14:06:14 2016 -0700

----------------------------------------------------------------------
 .../rel/rules/ReduceExpressionsRule.java        |  38 +++-
 .../java/org/apache/calcite/rex/RexUtil.java    | 215 ++++++++++++++++++-
 .../java/org/apache/calcite/sql/SqlKind.java    |  21 ++
 .../org/apache/calcite/tools/RelBuilder.java    |   2 +-
 .../apache/calcite/test/RelOptRulesTest.java    |  50 +++++
 .../org/apache/calcite/test/RexProgramTest.java |  62 ++++++
 .../org/apache/calcite/test/RelOptRulesTest.xml |  97 ++++++++-
 7 files changed, 480 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/3a54e54f/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 6c0238a..44870af 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -145,7 +145,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
       final RelMetadataQuery mq = RelMetadataQuery.instance();
       final RelOptPredicateList predicates =
           mq.getPulledUpPredicates(filter.getInput());
-      if (reduceExpressions(filter, expList, predicates)) {
+      if (reduceExpressions(filter, expList, predicates, true)) {
         assert expList.size() == 1;
         newConditionExp = expList.get(0);
         reduced = true;
@@ -435,6 +435,42 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
    */
   protected static boolean reduceExpressions(RelNode rel, List<RexNode> expList,
       RelOptPredicateList predicates) {
+    return reduceExpressions(rel, expList, predicates, false);
+  }
+
+  /**
+   * Reduces a list of expressions.
+   *
+   * @param rel     Relational expression
+   * @param expList List of expressions, modified in place
+   * @param predicates Constraints known to hold on input expressions
+   * @param unknownAsFalse Whether UNKNOWN will be treated as FALSE
+   *
+   * @return whether reduction found something to change, and succeeded
+   */
+  protected static boolean reduceExpressions(RelNode rel, List<RexNode> expList,
+      RelOptPredicateList predicates, boolean unknownAsFalse) {
+    RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+
+    boolean reduced = reduceExpressionsInternal(rel, expList, predicates);
+
+    // Simplify preds in place
+    boolean simplified = false;
+    for (int i = 0; i < expList.size(); i++) {
+      RexNode newExp = RexUtil.simplify(rexBuilder, expList.get(i),
+          unknownAsFalse);
+      if (!newExp.toString().equals(expList.get(i).toString())) {
+        expList.remove(i);
+        expList.add(i, newExp);
+        simplified = true;
+      }
+    }
+
+    return reduced || simplified;
+  }
+
+  protected static boolean reduceExpressionsInternal(RelNode rel, List<RexNode> expList,
+      RelOptPredicateList predicates) {
     RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
 
     // Replace predicates on CASE to CASE on predicates.

http://git-wip-us.apache.org/repos/asf/calcite/blob/3a54e54f/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 12ca400..86ffbfb 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -52,8 +52,11 @@ import com.google.common.collect.Sets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -306,6 +309,31 @@ public class RexUtil {
     return node.accept(ConstantFinder.INSTANCE);
   }
 
+  /**
+   * Returns whether a given expression is deterministic.
+   *
+   * @param e Expression
+   * @return true if tree result is deterministic, false otherwise
+   */
+  public static boolean isDeterministic(RexNode e) {
+    try {
+      RexVisitor<Void> visitor =
+          new RexVisitorImpl<Void>(true) {
+            @Override public Void visitCall(RexCall call) {
+              if (!call.getOperator().isDeterministic()) {
+                throw Util.FoundOne.NULL;
+              }
+              return null;
+            }
+          };
+      e.accept(visitor);
+      return true;
+    } catch (Util.FoundOne ex) {
+      Util.swallow(ex, null);
+      return false;
+    }
+  }
+
    /**
    * Returns whether a given node contains a RexCall with a specified operator
    *
@@ -1283,9 +1311,14 @@ public class RexUtil {
    * </ul>
    */
   public static RexNode simplify(RexBuilder rexBuilder, RexNode e) {
+    return simplify(rexBuilder, e, false);
+  }
+
+  public static RexNode simplify(RexBuilder rexBuilder, RexNode e,
+      boolean unknownAsFalse) {
     switch (e.getKind()) {
     case AND:
-      return simplifyAnd(rexBuilder, (RexCall) e);
+      return simplifyAnd(rexBuilder, (RexCall) e, unknownAsFalse);
     case OR:
       return simplifyOr(rexBuilder, (RexCall) e);
     case NOT:
@@ -1312,11 +1345,19 @@ public class RexUtil {
    */
   public static RexNode simplifyAnds(RexBuilder rexBuilder,
       Iterable<? extends RexNode> nodes) {
+    return simplifyAnds(rexBuilder, nodes, false);
+  }
+
+  public static RexNode simplifyAnds(RexBuilder rexBuilder,
+      Iterable<? extends RexNode> nodes, boolean unknownAsFalse) {
     final List<RexNode> terms = new ArrayList<>();
     final List<RexNode> notTerms = new ArrayList<>();
     for (RexNode e : nodes) {
       RelOptUtil.decomposeConjunction(e, terms, notTerms);
     }
+    if (unknownAsFalse) {
+      return simplifyAnd2ForUnknownAsFalse(rexBuilder, terms, notTerms);
+    }
     return simplifyAnd2(rexBuilder, terms, notTerms);
   }
 
@@ -1405,6 +1446,18 @@ public class RexUtil {
       return SqlStdOperatorTable.IS_NOT_TRUE;
     case IS_NOT_NULL:
       return SqlStdOperatorTable.IS_NOT_NULL;
+    case EQUALS:
+      return SqlStdOperatorTable.EQUALS;
+    case NOT_EQUALS:
+      return SqlStdOperatorTable.NOT_EQUALS;
+    case LESS_THAN:
+      return SqlStdOperatorTable.LESS_THAN;
+    case GREATER_THAN:
+      return SqlStdOperatorTable.GREATER_THAN;
+    case LESS_THAN_OR_EQUAL:
+      return SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
+    case GREATER_THAN_OR_EQUAL:
+      return SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
     default:
       throw new AssertionError(kind);
     }
@@ -1432,6 +1485,9 @@ public class RexUtil {
     assert newOperands.size() % 2 == 1;
     switch (newOperands.size()) {
     case 1:
+      if (!call.getType().equals(newOperands.get(0).getType())) {
+        return rexBuilder.makeCast(call.getType(), newOperands.get(0));
+      }
       return newOperands.get(0);
     }
   trueFalse:
@@ -1483,10 +1539,14 @@ public class RexUtil {
     return builder.build();
   }
 
-  public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e) {
+  public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e,
+      boolean unknownAsFalse) {
     final List<RexNode> terms = new ArrayList<>();
     final List<RexNode> notTerms = new ArrayList<>();
     RelOptUtil.decomposeConjunction(e, terms, notTerms);
+    if (unknownAsFalse) {
+      return simplifyAnd2ForUnknownAsFalse(rexBuilder, terms, notTerms);
+    }
     return simplifyAnd2(rexBuilder, terms, notTerms);
   }
 
@@ -1524,6 +1584,157 @@ public class RexUtil {
     return composeConjunction(rexBuilder, terms, false);
   }
 
+  /** As {@link #simplifyAnd2(RexBuilder, List, List)} but we assume that if the expression returns
+   * UNKNOWN it will be interpreted as FALSE. */
+  public static RexNode simplifyAnd2ForUnknownAsFalse(RexBuilder rexBuilder,
+      List<RexNode> terms, List<RexNode> notTerms) {
+    if (terms.contains(rexBuilder.makeLiteral(false))) {
+      return rexBuilder.makeLiteral(false);
+    }
+    if (terms.isEmpty() && notTerms.isEmpty()) {
+      return rexBuilder.makeLiteral(true);
+    }
+    if (terms.size() == 1 && notTerms.isEmpty()) {
+      // Make sure "x OR y OR x" (a single-term conjunction) gets simplified.
+      return simplify(rexBuilder, terms.get(0), true);
+    }
+    // Try to simplify the expression
+    final Set<String> negatedTerms = new HashSet<>();
+    final Set<String> nullOperands = new HashSet<>();
+    final Set<RexNode> notNullOperands = new LinkedHashSet<>();
+    final Set<String> comparedOperands = new HashSet<>();
+    for (int i = 0; i < terms.size(); i++) {
+      final RexNode term = terms.get(i);
+      if (!isDeterministic(term)) {
+        continue;
+      }
+      switch (term.getKind()) {
+      case EQUALS:
+      case NOT_EQUALS:
+      case LESS_THAN:
+      case GREATER_THAN:
+      case LESS_THAN_OR_EQUAL:
+      case GREATER_THAN_OR_EQUAL:
+        RexCall call = (RexCall) term;
+        RexNode left = call.getOperands().get(0);
+        comparedOperands.add(left.toString());
+        // if it is a cast, we include the inner reference
+        if (left.getKind() == SqlKind.CAST) {
+          RexCall leftCast = (RexCall) left;
+          comparedOperands.add(leftCast.getOperands().get(0).toString());
+        }
+        RexNode right = call.getOperands().get(1);
+        comparedOperands.add(right.toString());
+        // if it is a cast, we include the inner reference
+        if (right.getKind() == SqlKind.CAST) {
+          RexCall rightCast = (RexCall) right;
+          comparedOperands.add(rightCast.getOperands().get(0).toString());
+        }
+        // Assume the expression a > 5 is part of a Filter condition.
+        // Then we can derive the negated term: a <= 5.
+        // But as the comparison is string based and thus operands order dependent,
+        // we should also add the inverted negated term: 5 >= a.
+        // Observe that for creating the inverted term we invert the list of operands.
+        RexNode negatedTerm = negate(rexBuilder, call);
+        if (negatedTerm != null) {
+          negatedTerms.add(negatedTerm.toString());
+          RexNode invertNegatedTerm = invert(rexBuilder, (RexCall) negatedTerm);
+          if (invertNegatedTerm != null) {
+            negatedTerms.add(invertNegatedTerm.toString());
+          }
+        }
+        break;
+      case IN:
+        comparedOperands.add(((RexCall) term).operands.get(0).toString());
+        break;
+      case BETWEEN:
+        comparedOperands.add(((RexCall) term).operands.get(1).toString());
+        break;
+      case IS_NOT_NULL:
+        notNullOperands.add(((RexCall) term).getOperands().get(0));
+        terms.remove(i);
+        --i;
+        break;
+      case IS_NULL:
+        nullOperands.add(((RexCall) term).getOperands().get(0).toString());
+      }
+    }
+    // If one column should be null and is in a comparison predicate,
+    // it is not satisfiable.
+    // Example. IS NULL(x) AND x < 5  - not satisfiable
+    if (!Collections.disjoint(nullOperands, comparedOperands)) {
+      return rexBuilder.makeLiteral(false);
+    }
+    // Remove not necessary IS NOT NULL expressions.
+    //
+    // Example. IS NOT NULL(x) AND x < 5  : x < 5
+    for (RexNode operand : notNullOperands) {
+      if (!comparedOperands.contains(operand.toString())) {
+        terms.add(
+            rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, operand));
+      }
+    }
+    // If one of the not-disjunctions is a disjunction that is wholly
+    // contained in the disjunctions list, the expression is not
+    // satisfiable.
+    //
+    // Example #1. x AND y AND z AND NOT (x AND y)  - not satisfiable
+    // Example #2. x AND y AND NOT (x AND y)        - not satisfiable
+    // Example #3. x AND y AND NOT (x AND y AND z)  - may be satisfiable
+    final Set<String> termsSet = Sets.newHashSet(RexUtil.strings(terms));
+    for (RexNode notDisjunction : notTerms) {
+      if (!isDeterministic(notDisjunction)) {
+        continue;
+      }
+      final List<String> terms2Set = RexUtil.strings(
+              RelOptUtil.conjunctions(notDisjunction));
+      if (termsSet.containsAll(terms2Set)) {
+        return rexBuilder.makeLiteral(false);
+      }
+    }
+    // Add the NOT disjunctions back in.
+    for (RexNode notDisjunction : notTerms) {
+      terms.add(
+          simplify(rexBuilder,
+              rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction), true));
+    }
+    // The negated terms: only deterministic expressions
+    for (String negatedTerm : negatedTerms) {
+      if (termsSet.contains(negatedTerm)) {
+        return rexBuilder.makeLiteral(false);
+      }
+    }
+    return composeConjunction(rexBuilder, terms, false);
+  }
+
+  public static RexNode negate(RexBuilder rexBuilder, RexCall call) {
+    switch (call.getKind()) {
+    case EQUALS:
+    case NOT_EQUALS:
+    case LESS_THAN:
+    case GREATER_THAN:
+    case LESS_THAN_OR_EQUAL:
+    case GREATER_THAN_OR_EQUAL:
+      final SqlOperator op = op(call.getKind().negateNullSafe());
+      return rexBuilder.makeCall(op, call.getOperands());
+    }
+    return null;
+  }
+
+  public static RexNode invert(RexBuilder rexBuilder, RexCall call) {
+    switch (call.getKind()) {
+    case EQUALS:
+    case NOT_EQUALS:
+    case LESS_THAN:
+    case GREATER_THAN:
+    case LESS_THAN_OR_EQUAL:
+    case GREATER_THAN_OR_EQUAL:
+      final SqlOperator op = op(call.getKind().reverse());
+      return rexBuilder.makeCall(op, Lists.reverse(call.getOperands()));
+    }
+    return null;
+  }
+
   /** Simplifies OR(x, x) into x, and similar. */
   public static RexNode simplifyOr(RexBuilder rexBuilder, RexCall call) {
     assert call.getKind() == SqlKind.OR;

http://git-wip-us.apache.org/repos/asf/calcite/blob/3a54e54f/core/src/main/java/org/apache/calcite/sql/SqlKind.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 07d2bca..bb84547 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -999,6 +999,27 @@ public enum SqlKind {
     }
   }
 
+  /** Returns the kind that you get if you negate this kind.
+   * To conform to null semantics, null value should not be compared. */
+  public SqlKind negateNullSafe() {
+    switch (this) {
+    case EQUALS:
+      return NOT_EQUALS;
+    case NOT_EQUALS:
+      return EQUALS;
+    case LESS_THAN:
+      return GREATER_THAN_OR_EQUAL;
+    case GREATER_THAN:
+      return LESS_THAN_OR_EQUAL;
+    case LESS_THAN_OR_EQUAL:
+      return GREATER_THAN;
+    case GREATER_THAN_OR_EQUAL:
+      return LESS_THAN;
+    default:
+      return this.negate();
+    }
+  }
+
   /**
    * Returns whether this {@code SqlKind} belongs to a given category.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/3a54e54f/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 54398a6..414f1ca 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -697,7 +697,7 @@ public class RelBuilder {
    * and optimized in a similar way to the {@link #and} method.
    * If the result is TRUE no filter is created. */
   public RelBuilder filter(Iterable<? extends RexNode> predicates) {
-    final RexNode x = RexUtil.simplifyAnds(cluster.getRexBuilder(), predicates);
+    final RexNode x = RexUtil.simplifyAnds(cluster.getRexBuilder(), predicates, true);
     if (x.isAlwaysFalse()) {
       return empty();
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/3a54e54f/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 9a2c3db..f28f4e0 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -1227,6 +1227,56 @@ public class RelOptRulesTest extends RelOptTestBase {
             + " where a - b < 21");
   }
 
+  @Test public void testReduceCase() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
+        .build();
+
+    final String sql = "select\n"
+        + "  case when false then cast(2.1 as float)\n"
+        + "   else cast(1 as integer) end as newcol\n"
+        + "from emp";
+    checkPlanning(program, sql);
+  }
+
+  @Test public void testReduceConstantsIsNull() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .build();
+
+    checkPlanning(program,
+        "select empno from emp where empno=10 and empno is null");
+  }
+
+  @Test public void testReduceConstantsIsNotNull() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .build();
+
+    final String sql = "select empno from emp\n"
+        + "where empno=10 and empno is not null";
+    checkPlanning(program, sql);
+  }
+
+  @Test public void testReduceConstantsNegated() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .build();
+
+    final String sql = "select empno from emp\n"
+        + "where empno=10 and not(empno=10)";
+    checkPlanning(program, sql);
+  }
+
+  @Test public void testReduceConstantsNegatedInverted() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .build();
+
+    final String sql = "select empno from emp where empno>10 and empno<=10";
+    checkPlanning(program, sql);
+  }
+
   @Ignore // Calcite does not support INSERT yet
   @Test public void testReduceValuesNull() throws Exception {
     // The NULL literal presents pitfalls for value-reduction. Only

http://git-wip-us.apache.org/repos/asf/calcite/blob/3a54e54f/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index 1efb8cc..ed56f90 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -94,6 +94,11 @@ public class RexProgramTest {
         equalTo(expected));
   }
 
+  private void checkSimplifyFilter(RexNode node, String expected) {
+    assertThat(RexUtil.simplify(rexBuilder, node, true).toString(),
+        equalTo(expected));
+  }
+
   /** Returns the number of nodes (including leaves) in a Rex tree. */
   private static int nodeCount(RexNode node) {
     int n = 1;
@@ -137,6 +142,22 @@ public class RexProgramTest {
     return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, n1, n2);
   }
 
+  private RexNode le(RexNode n1, RexNode n2) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, n1, n2);
+  }
+
+  private RexNode lt(RexNode n1, RexNode n2) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, n1, n2);
+  }
+
+  private RexNode ge(RexNode n1, RexNode n2) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, n1, n2);
+  }
+
+  private RexNode gt(RexNode n1, RexNode n2) {
+    return rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, n1, n2);
+  }
+
   /**
    * Tests construction of a RexProgram.
    */
@@ -759,6 +780,7 @@ public class RexProgramTest {
     final RexNode eRef = rexBuilder.makeFieldAccess(range, 4);
     final RexLiteral true_ = rexBuilder.makeLiteral(true);
     final RexLiteral false_ = rexBuilder.makeLiteral(false);
+    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
 
     // and: remove duplicates
     checkSimplify(and(aRef, bRef, aRef), "AND(?0.a, ?0.b)");
@@ -830,6 +852,46 @@ public class RexProgramTest {
     // is not null, applied to not-null value
     checkSimplify(rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, aRef),
         "true");
+
+    // condition, and the inverse - nothing to do due to null values
+    checkSimplify(and(le(aRef, literal1), gt(aRef, literal1)),
+        "AND(<=(?0.a, 1), >(?0.a, 1))");
+
+    checkSimplify(and(le(aRef, literal1), ge(aRef, literal1)),
+        "AND(<=(?0.a, 1), >=(?0.a, 1))");
+
+    checkSimplify(and(lt(aRef, literal1), eq(aRef, literal1), ge(aRef, literal1)),
+        "AND(<(?0.a, 1), =(?0.a, 1), >=(?0.a, 1))");
+  }
+
+  @Test public void testSimplifyFilter() {
+    final RelDataType booleanType =
+        typeFactory.createSqlType(SqlTypeName.BOOLEAN);
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    final RelDataType rowType = typeFactory.builder()
+        .add("a", booleanType)
+        .add("b", booleanType)
+        .add("c", booleanType)
+        .add("d", booleanType)
+        .add("e", booleanType)
+        .add("f", booleanType)
+        .add("g", booleanType)
+        .add("h", intType)
+        .build();
+
+    final RexDynamicParam range = rexBuilder.makeDynamicParam(rowType, 0);
+    final RexNode aRef = rexBuilder.makeFieldAccess(range, 0);
+    final RexLiteral literal1 = rexBuilder.makeExactLiteral(BigDecimal.ONE);
+
+    // condition, and the inverse
+    checkSimplifyFilter(and(le(aRef, literal1), gt(aRef, literal1)),
+        "false");
+
+    checkSimplifyFilter(and(le(aRef, literal1), ge(aRef, literal1)),
+        "AND(<=(?0.a, 1), >=(?0.a, 1))");
+
+    checkSimplifyFilter(and(lt(aRef, literal1), eq(aRef, literal1), ge(aRef, literal1)),
+        "false");
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/3a54e54f/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index c138c3f..1600c7e 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -3606,6 +3606,26 @@ LogicalProject(DEPTNO=[$0], ENAME=[$1])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testReduceCase">
+        <Resource name="sql">
+            <![CDATA[select
+  case when false then cast(2.1 as float)
+   else cast(1 as integer) end as newcol
+from emp]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(NEWCOL=[CASE(false, 2.1, 1)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(NEWCOL=[1E0])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testReduceConstantsRequiresExecutor">
         <Resource name="sql">
             <![CDATA[select * from (values (1,2)) where 1 + 2 > 3 + CAST(NULL AS INTEGER)]]>
@@ -3763,6 +3783,81 @@ LogicalProject(EMPNO=[$0], EXPR$1=[$2])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testReduceConstantsIsNotNull">
+        <Resource name="sql">
+            <![CDATA[select empno from emp
+where empno=10 and empno is not null]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[AND(=($0, 10), IS NOT NULL($0))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[=($0, 10)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testReduceConstantsIsNull">
+        <Resource name="sql">
+            <![CDATA[select empno from emp where empno=10 and empno is null]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[AND(=($0, 10), IS NULL($0))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalValues(tuples=[[]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testReduceConstantsNegated">
+        <Resource name="sql">
+            <![CDATA[select empno from emp
+where empno=10 and not(empno=10)]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[AND(=($0, 10), NOT(=($0, 10)))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalValues(tuples=[[]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testReduceConstantsNegatedInverted">
+        <Resource name="sql">
+            <![CDATA[select empno from emp where empno>10 and empno<=10]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalFilter(condition=[AND(>($0, 10), <=($0, 10))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0])
+  LogicalValues(tuples=[[]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testSwapOuterJoin">
         <Resource name="sql">
             <![CDATA[select 1 from sales.dept d left outer join sales.emp e on d.deptno = e.deptno]]>
@@ -4732,7 +4827,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[10], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  LogicalFilter(condition=[AND(=($7, 7), =($7, 8), =($0, 10), IS NULL($3), =($0, 10))])
+  LogicalFilter(condition=[AND(=($7, 7), =($7, 8), =($0, 10), IS NULL($3))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>


[2/2] calcite git commit: Add test case for [DRILL-4407] GROUP BY subquery causes Java NPE

Posted by jh...@apache.org.
Add test case for [DRILL-4407] GROUP BY subquery causes Java NPE


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/0ad58ed6
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/0ad58ed6
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/0ad58ed6

Branch: refs/heads/master
Commit: 0ad58ed624dfad43bfcd26881588a9d3482397c1
Parents: 3a54e54
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Feb 17 13:33:09 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Mar 17 15:15:42 2016 -0700

----------------------------------------------------------------------
 core/src/test/resources/sql/subquery.iq | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/0ad58ed6/core/src/test/resources/sql/subquery.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/subquery.iq b/core/src/test/resources/sql/subquery.iq
index 8f0eb70..d4f3a10 100644
--- a/core/src/test/resources/sql/subquery.iq
+++ b/core/src/test/resources/sql/subquery.iq
@@ -274,6 +274,17 @@ GROUP BY emp.deptno;
 
 !ok
 
+# [DRILL-4407] Group by subquery causes Java NPE
+select count(*) as c
+from "scott".emp
+group by (select deptno from "scott".emp where empno = 10);
+ C
+----
+ 14
+(1 row)
+
+!ok
+
 !if (fixed.calcite1045) {
 # Correlated IN sub-query in WHERE clause of JOIN
 select empno from "scott".emp as e