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/01/08 02:08:29 UTC

[6/6] calcite git commit: [CALCITE-1038] Obsolete AggregateConstantKeyRule

[CALCITE-1038] Obsolete AggregateConstantKeyRule

Move AggregateConstantKeyRule's functionality into
AggregateProjectPullUpConstantsRule and obsolete it.

Detect constants using metadata (RelMdPredicates). The input does not
need to be a Project, and constants do not need to be literals.


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

Branch: refs/heads/master
Commit: 904c73da60b9f9deec61ea34d89ada3462381f93
Parents: 8531ab1
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 4 10:35:43 2016 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 6 02:37:31 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     |   4 +-
 .../rel/rules/AggregateConstantKeyRule.java     | 130 ---------------
 .../AggregateProjectPullUpConstantsRule.java    | 163 +++++++++----------
 .../rel/rules/ReduceExpressionsRule.java        |  67 +++++---
 .../apache/calcite/test/RelOptRulesTest.java    |  15 +-
 .../org/apache/calcite/test/RelOptRulesTest.xml |  14 +-
 6 files changed, 135 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index e658749..e444612 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -39,7 +39,6 @@ import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.rules.AggregateConstantKeyRule;
 import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
 import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.calcite.rel.rules.MultiJoin;
@@ -1546,8 +1545,7 @@ public abstract class RelOptUtil {
   }
 
   public static void registerAbstractRels(RelOptPlanner planner) {
-    planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE);
-    planner.addRule(AggregateConstantKeyRule.INSTANCE);
+    planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE2);
     planner.addRule(PruneEmptyRules.UNION_INSTANCE);
     planner.addRule(PruneEmptyRules.PROJECT_INSTANCE);
     planner.addRule(PruneEmptyRules.FILTER_INSTANCE);

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
deleted file mode 100644
index ea31178..0000000
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateConstantKeyRule.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.rel.rules;
-
-import org.apache.calcite.plan.RelOptPredicateList;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.ImmutableBitSet;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-/**
- * Planner rule that removes constant keys from an
- * a {@link Aggregate}.
- *
- * <p>It never removes the last column, because {@code Aggregate([])} returns
- * 1 row even if its input is empty.
- */
-public class AggregateConstantKeyRule extends RelOptRule {
-  public static final AggregateConstantKeyRule INSTANCE =
-      new AggregateConstantKeyRule(RelFactories.LOGICAL_BUILDER,
-          "AggregateConstantKeyRule");
-
-  //~ Constructors -----------------------------------------------------------
-
-  /** Creates an AggregateConstantKeyRule. */
-  private AggregateConstantKeyRule(RelBuilderFactory relBuilderFactory,
-      String description) {
-    super(operand(Aggregate.class, null, Aggregate.IS_SIMPLE, any()),
-        relBuilderFactory, description);
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  public void onMatch(RelOptRuleCall call) {
-    final Aggregate aggregate = call.rel(0);
-    assert !aggregate.indicator : "predicate ensured no grouping sets";
-
-    final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
-    final RelOptPredicateList predicates =
-        RelMetadataQuery.getPulledUpPredicates(aggregate.getInput());
-    if (predicates == null) {
-      return;
-    }
-    final ImmutableMap<RexNode, RexLiteral> constants =
-        ReduceExpressionsRule.predicateConstants(rexBuilder, predicates);
-    final NavigableMap<Integer, RexLiteral> map = new TreeMap<>();
-    for (int key : aggregate.getGroupSet()) {
-      final RexInputRef ref =
-          rexBuilder.makeInputRef(aggregate.getInput(), key);
-      if (constants.containsKey(ref)) {
-        map.put(key, constants.get(ref));
-      }
-    }
-
-    if (map.isEmpty()) {
-      return; // none of the keys are constant
-    }
-
-    if (map.size() == aggregate.getGroupCount()) {
-      if (map.size() == 1) {
-        // There is one key, and it is constant. We cannot remove it.
-        return;
-      }
-      map.remove(map.descendingKeySet().descendingIterator().next());
-    }
-
-    ImmutableBitSet newGroupSet = aggregate.getGroupSet();
-    for (int key : map.keySet()) {
-      newGroupSet = newGroupSet.clear(key);
-    }
-    final Aggregate newAggregate =
-        aggregate.copy(aggregate.getTraitSet(), aggregate.getInput(),
-            false, newGroupSet, ImmutableList.of(newGroupSet),
-            aggregate.getAggCallList());
-    final RelBuilder relBuilder = call.builder();
-    relBuilder.push(newAggregate);
-
-    final List<RexNode> projects = new ArrayList<>();
-    int offset = 0;
-    for (RelDataTypeField field : aggregate.getRowType().getFieldList()) {
-      RexNode node = null;
-      if (field.getIndex() < aggregate.getGroupCount()) {
-        node = map.get(aggregate.getGroupSet().nth(field.getIndex()));
-        if (node != null) {
-          node = relBuilder.getRexBuilder().makeCast(field.getType(), node, true);
-          node = relBuilder.alias(node, field.getName());
-          ++offset;
-        }
-      }
-      if (node == null) {
-        node = relBuilder.field(field.getIndex() - offset);
-      }
-      projects.add(node);
-    }
-    call.transformTo(relBuilder.project(projects).build());
-  }
-}
-
-// End AggregateConstantKeyRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
index 2b99cf6..a0ff130 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateProjectPullUpConstantsRule.java
@@ -16,70 +16,91 @@
  */
 package org.apache.calcite.rel.rules;
 
+import org.apache.calcite.plan.RelOptPredicateList;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
-import org.apache.calcite.util.mapping.Mapping;
-import org.apache.calcite.util.mapping.MappingType;
+
+import com.google.common.collect.ImmutableMap;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
 /**
- * Planner rule that removes constant expressions from the
- * group list of an {@link org.apache.calcite.rel.logical.LogicalAggregate}.
+ * Planner rule that removes constant keys from an
+ * {@link org.apache.calcite.rel.core.Aggregate}.
+ *
+ * <p>Constant fields are deduced using
+ * {@link RelMetadataQuery#getPulledUpPredicates(RelNode)}; the input does not
+ * need to be a {@link org.apache.calcite.rel.core.Project}.
  *
- * <p><b>Effect of the rule</b></p>
+ * <p>This rules never removes the last column, because {@code Aggregate([])}
+ * returns 1 row even if its input is empty.
  *
  * <p>Since the transformed relational expression has to match the original
  * relational expression, the constants are placed in a projection above the
  * reduced aggregate. If those constants are not used, another rule will remove
  * them from the project.
- *
- * <p>LogicalAggregate needs its group columns to be on the prefix of its input
- * relational expression. Therefore, if a constant is not on the trailing edge
- * of the group list, removing it will leave a hole. In this case, the rule adds
- * a project before the aggregate to reorder the columns, and permutes them back
- * afterwards.
  */
 public class AggregateProjectPullUpConstantsRule extends RelOptRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
   public static final AggregateProjectPullUpConstantsRule INSTANCE =
-      new AggregateProjectPullUpConstantsRule();
+      new AggregateProjectPullUpConstantsRule(LogicalAggregate.class,
+          LogicalProject.class, RelFactories.LOGICAL_BUILDER,
+          "AggregateProjectPullUpConstantsRule");
+
+  /** More general instance that matches any relational expression. */
+  public static final AggregateProjectPullUpConstantsRule INSTANCE2 =
+      new AggregateProjectPullUpConstantsRule(LogicalAggregate.class,
+          RelNode.class, RelFactories.LOGICAL_BUILDER,
+          "AggregatePullUpConstantsRule");
 
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Private: use singleton
+   * Creates an AggregateProjectPullUpConstantsRule.
+   *
+   * @param aggregateClass Aggregate class
+   * @param inputClass Input class, such as {@link LogicalProject}
+   * @param relBuilderFactory Builder for relational expressions
+   * @param description Description, or null to guess description
    */
-  private AggregateProjectPullUpConstantsRule() {
+  public AggregateProjectPullUpConstantsRule(
+      Class<? extends Aggregate> aggregateClass,
+      Class<? extends RelNode> inputClass,
+      RelBuilderFactory relBuilderFactory, String description) {
     super(
-        operand(LogicalAggregate.class, null, Aggregate.IS_SIMPLE,
-            operand(LogicalProject.class, any())));
+        operand(aggregateClass, null, Aggregate.IS_SIMPLE,
+            operand(inputClass, any())),
+        relBuilderFactory, description);
   }
 
   //~ Methods ----------------------------------------------------------------
 
   public void onMatch(RelOptRuleCall call) {
-    final LogicalAggregate aggregate = call.rel(0);
-    final LogicalProject input = call.rel(1);
+    final Aggregate aggregate = call.rel(0);
+    final RelNode input = call.rel(1);
 
+    assert !aggregate.indicator : "predicate ensured no grouping sets";
     final int groupCount = aggregate.getGroupCount();
     if (groupCount == 1) {
       // No room for optimization since we cannot convert from non-empty
@@ -87,47 +108,49 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       return;
     }
 
-    final RexProgram program =
-      RexProgram.create(input.getInput().getRowType(),
-          input.getProjects(),
-          null,
-          input.getRowType(),
-          input.getCluster().getRexBuilder());
-
-    final RelDataType childRowType = input.getRowType();
-    final List<Integer> constantList = new ArrayList<>();
-    final Map<Integer, RexNode> constants = new HashMap<>();
-    for (int i : aggregate.getGroupSet()) {
-      final RexLocalRef ref = program.getProjectList().get(i);
-      if (program.isConstant(ref)) {
-        constantList.add(i);
-        constants.put(
-            i,
-            program.gatherExpr(ref));
+    final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder();
+    final RelOptPredicateList predicates =
+        RelMetadataQuery.getPulledUpPredicates(aggregate.getInput());
+    if (predicates == null) {
+      return;
+    }
+    final ImmutableMap<RexNode, RexNode> constants =
+        ReduceExpressionsRule.predicateConstants(RexNode.class, rexBuilder,
+            predicates);
+    final NavigableMap<Integer, RexNode> map = new TreeMap<>();
+    for (int key : aggregate.getGroupSet()) {
+      final RexInputRef ref =
+          rexBuilder.makeInputRef(aggregate.getInput(), key);
+      if (constants.containsKey(ref)) {
+        map.put(key, constants.get(ref));
       }
     }
 
     // None of the group expressions are constant. Nothing to do.
-    if (constantList.size() == 0) {
+    if (map.isEmpty()) {
       return;
     }
 
-    if (groupCount == constantList.size()) {
+    if (groupCount == map.size()) {
       // At least a single item in group by is required.
-      // Otherwise group by 1,2 might be altered to group by ()
+      // Otherwise "GROUP BY 1, 2" might be altered to "GROUP BY ()".
       // Removing of the first element is not optimal here,
       // however it will allow us to use fast path below (just trim
-      // groupCount)
-      constantList.remove(0);
+      // groupCount).
+      map.remove(map.navigableKeySet().first());
     }
 
-    final int newGroupCount = groupCount - constantList.size();
+    ImmutableBitSet newGroupSet = aggregate.getGroupSet();
+    for (int key : map.keySet()) {
+      newGroupSet = newGroupSet.clear(key);
+    }
+    final int newGroupCount = newGroupSet.cardinality();
 
     // If the constants are on the trailing edge of the group list, we just
     // reduce the group count.
     final RelBuilder relBuilder = call.builder();
     relBuilder.push(input);
-    if (constantList.get(0) == newGroupCount) {
+    if (map.navigableKeySet().first() == newGroupCount) {
       // Clone aggregate calls.
       final List<AggregateCall> newAggCalls = new ArrayList<>();
       for (AggregateCall aggCall : aggregate.getAggCallList()) {
@@ -136,13 +159,13 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
                 groupCount, newGroupCount));
       }
       relBuilder.aggregate(
-          relBuilder.groupKey(ImmutableBitSet.range(newGroupCount), false, null),
+          relBuilder.groupKey(newGroupSet, false, null),
           newAggCalls);
     } else {
       // Create the mapping from old field positions to new field
       // positions.
       final Permutation mapping =
-          new Permutation(childRowType.getFieldCount());
+          new Permutation(input.getRowType().getFieldCount());
       mapping.identity();
 
       // Ensure that the first positions in the mapping are for the new
@@ -150,18 +173,13 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       for (int i = 0, groupOrdinal = 0, constOrdinal = newGroupCount;
           i < groupCount;
           ++i) {
-        if (i >= groupCount) {
-          mapping.set(i, i);
-        } else if (constants.containsKey(i)) {
+        if (map.containsKey(i)) {
           mapping.set(i, constOrdinal++);
         } else {
           mapping.set(i, groupOrdinal++);
         }
       }
 
-      // Create a projection to permute fields into these positions.
-      createProjection(relBuilder, mapping);
-
       // Adjust aggregate calls for new field positions.
       final List<AggregateCall> newAggCalls = new ArrayList<>();
       for (AggregateCall aggCall : aggregate.getAggCallList()) {
@@ -180,7 +198,7 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
 
       // Aggregate on projection.
       relBuilder.aggregate(
-          relBuilder.groupKey(ImmutableBitSet.range(newGroupCount), false, null),
+          relBuilder.groupKey(newGroupSet, false, null),
               newAggCalls);
     }
 
@@ -192,10 +210,10 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
       final int i = field.getIndex();
       if (i >= groupCount) {
         // Aggregate expressions' names and positions are unchanged.
-        expr = relBuilder.field(i - constantList.size());
-      } else if (constantList.contains(i)) {
+        expr = relBuilder.field(i - map.size());
+      } else if (map.containsKey(i)) {
         // Re-generate the constant expression in the project.
-        expr = constants.get(i);
+        expr = map.get(i);
       } else {
         // Project the aggregation expression, in its original
         // position.
@@ -208,33 +226,6 @@ public class AggregateProjectPullUpConstantsRule extends RelOptRule {
     call.transformTo(relBuilder.build());
   }
 
-  /**
-   * Creates a projection which permutes the fields of a given relational
-   * expression.
-   *
-   * <p>For example, given a relational expression [A, B, C, D] and a mapping
-   * [2:1, 3:0], returns a projection [$3 AS C, $2 AS B].
-   *
-   * @param relBuilder Relational expression builder
-   * @param mapping Mapping to apply to source columns
-   */
-  private static RelBuilder createProjection(RelBuilder relBuilder,
-      Mapping mapping) {
-    // Every target has precisely one source; every source has at most
-    // one target.
-    assert mapping.getMappingType().isA(MappingType.INVERSE_SURJECTION);
-    final RelDataType childRowType = relBuilder.peek().getRowType();
-    assert mapping.getSourceCount() == childRowType.getFieldCount();
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
-    for (int target = 0; target < mapping.getTargetCount(); ++target) {
-      int source = mapping.getSource(target);
-      projects.add(
-          Pair.<RexNode, String>of(
-              relBuilder.field(source),
-              childRowType.getFieldList().get(source).getName()));
-    }
-    return relBuilder.project(Pair.left(projects), Pair.right(projects));
-  }
 }
 
 // End AggregateProjectPullUpConstantsRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/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 8be202e..17d5cb0 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
@@ -396,7 +396,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     List<Boolean> addCasts = Lists.newArrayList();
     final List<RexNode> removableCasts = Lists.newArrayList();
     final ImmutableMap<RexNode, RexLiteral> constants =
-        predicateConstants(rexBuilder, predicates);
+        predicateConstants(RexLiteral.class, rexBuilder, predicates);
     findReducibleExps(rel.getCluster().getTypeFactory(), expList, constants,
         constExps, addCasts, removableCasts);
     if (constExps.isEmpty() && removableCasts.isEmpty()) {
@@ -495,7 +495,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
    * @param removableCasts returns the list of cast expressions where the cast
    */
   protected static void findReducibleExps(RelDataTypeFactory typeFactory,
-      List<RexNode> exps, ImmutableMap<RexNode, RexLiteral> constants,
+      List<RexNode> exps, ImmutableMap<RexNode, ? extends RexNode> constants,
       List<RexNode> constExps, List<Boolean> addCasts,
       List<RexNode> removableCasts) {
     ReducibleExprLocator gardener =
@@ -507,7 +507,19 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     assert constExps.size() == addCasts.size();
   }
 
-  protected static ImmutableMap<RexNode, RexLiteral> predicateConstants(
+  /** Creates a map containing each (e, constant) pair that occurs within
+   * a predicate list.
+   *
+   * @param clazz Class of expression that is considered constant
+   * @param rexBuilder Rex builder
+   * @param predicates Predicate list
+   * @param <C> what to consider a constant: {@link RexLiteral} to use a narrow
+   *           definition of constant, or {@link RexNode} to use
+   *           {@link RexUtil#isConstant(RexNode)}
+   * @return Map from values to constants
+   */
+  protected static <C extends RexNode> ImmutableMap<RexNode, C>
+  predicateConstants(Class<C> clazz,
       RexBuilder rexBuilder, RelOptPredicateList predicates) {
     // We cannot use an ImmutableMap.Builder here. If there are multiple entries
     // with the same key (e.g. "WHERE deptno = 1 AND deptno = 2"), it doesn't
@@ -518,14 +530,14 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     // (e.g. "WHERE deptno = 1 AND deptno = 1")
     // (3) It will return false if there are inconsistent constraints (e.g.
     // "WHERE deptno = 1 AND deptno = 2")
-    final Map<RexNode, RexLiteral> map = new HashMap<>();
+    final Map<RexNode, C> map = new HashMap<>();
     final Set<RexNode> excludeSet = new HashSet<>();
     for (RexNode predicate : predicates.pulledUpPredicates) {
-      gatherConstraints(predicate, map, excludeSet, rexBuilder);
+      gatherConstraints(clazz, predicate, map, excludeSet, rexBuilder);
     }
-    final ImmutableMap.Builder<RexNode, RexLiteral> builder =
+    final ImmutableMap.Builder<RexNode, C> builder =
         ImmutableMap.builder();
-    for (Map.Entry<RexNode, RexLiteral> entry : map.entrySet()) {
+    for (Map.Entry<RexNode, C> entry : map.entrySet()) {
       RexNode rexNode = entry.getKey();
       if (!overlap(rexNode, excludeSet)) {
         builder.put(rexNode, entry.getValue());
@@ -559,8 +571,8 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     }
   }
 
-  private static void gatherConstraints(RexNode predicate,
-      Map<RexNode, RexLiteral> map, Set<RexNode> excludeSet,
+  private static <C extends RexNode> void gatherConstraints(Class<C> clazz,
+      RexNode predicate, Map<RexNode, C> map, Set<RexNode> excludeSet,
       RexBuilder rexBuilder) {
     if (predicate.getKind() != SqlKind.EQUALS) {
       decompose(excludeSet, predicate);
@@ -576,21 +588,24 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     final RexNode right = operands.get(1);
     // note that literals are immutable too and they can only be compared through
     // values.
-    if (right instanceof RexLiteral) {
-      foo(left, (RexLiteral) right, map, excludeSet, rexBuilder);
-    }
-    if (left instanceof RexLiteral) {
-      foo(right, (RexLiteral) left, map, excludeSet, rexBuilder);
-    }
+    gatherConstraint(clazz, left, right, map, excludeSet, rexBuilder);
+    gatherConstraint(clazz, right, left, map, excludeSet, rexBuilder);
   }
 
-  private static void foo(RexNode left, RexLiteral right,
-      Map<RexNode, RexLiteral> map, Set<RexNode> excludeSet,
+  private static <C extends RexNode> void gatherConstraint(Class<C> clazz,
+      RexNode left, RexNode right, Map<RexNode, C> map, Set<RexNode> excludeSet,
       RexBuilder rexBuilder) {
+    if (!clazz.isInstance(right)) {
+      return;
+    }
+    if (!RexUtil.isConstant(right)) {
+      return;
+    }
+    C constant = clazz.cast(right);
     if (excludeSet.contains(left)) {
       return;
     }
-    final RexLiteral existedValue = map.get(left);
+    final C existedValue = map.get(left);
     if (existedValue == null) {
       switch (left.getKind()) {
       case CAST:
@@ -599,16 +614,19 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
         final RexNode operand = ((RexCall) left).getOperands().get(0);
         if (canAssignFrom(left.getType(), operand.getType())) {
           final RexNode castRight =
-              rexBuilder.makeCast(operand.getType(), right);
+              rexBuilder.makeCast(operand.getType(), constant);
           if (castRight instanceof RexLiteral) {
             left = operand;
-            right = (RexLiteral) castRight;
+            constant = clazz.cast(castRight);
           }
         }
       }
-      map.put(left, right);
+      map.put(left, constant);
     } else {
-      if (!existedValue.getValue().equals(right.getValue())) {
+      if (existedValue instanceof RexLiteral
+          && constant instanceof RexLiteral
+          && !((RexLiteral) existedValue).getValue()
+              .equals(((RexLiteral) constant).getValue())) {
         // we found conflicting values, e.g. left = 10 and left = 20
         map.remove(left);
         excludeSet.add(left);
@@ -770,7 +788,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
 
     private final List<Constancy> stack;
 
-    private final ImmutableMap<RexNode, RexLiteral> constants;
+    private final ImmutableMap<RexNode, ? extends RexNode> constants;
 
     private final List<RexNode> constExprs;
 
@@ -781,7 +799,8 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
     private final List<SqlOperator> parentCallTypeStack;
 
     ReducibleExprLocator(RelDataTypeFactory typeFactory,
-        ImmutableMap<RexNode, RexLiteral> constants, List<RexNode> constExprs,
+        ImmutableMap<RexNode, ? extends RexNode> constants,
+        List<RexNode> constExprs,
         List<Boolean> addCasts, List<RexNode> removableCasts) {
       // go deep
       super(true);

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/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 4061b0d..92bae3e 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -36,7 +36,6 @@ import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
-import org.apache.calcite.rel.rules.AggregateConstantKeyRule;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateFilterTransposeRule;
 import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
@@ -2027,7 +2026,7 @@ public class RelOptRulesTest extends RelOptTestBase {
    * Planner rule that removes Aggregate keys that are constant</a>. */
   @Test public void testAggregateConstantKeyRule() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select count(*) as c\n"
         + "from sales.emp\n"
@@ -2036,11 +2035,11 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanning(new HepPlanner(program), sql);
   }
 
-  /** Tests {@link AggregateConstantKeyRule} where reduction is not possible
-   * because "deptno" is the only key. */
+  /** Tests {@link AggregateProjectPullUpConstantsRule} where reduction is not
+   * possible because "deptno" is the only key. */
   @Test public void testAggregateConstantKeyRule2() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select count(*) as c\n"
         + "from sales.emp\n"
@@ -2049,11 +2048,11 @@ public class RelOptRulesTest extends RelOptTestBase {
     checkPlanUnchanged(new HepPlanner(program), sql);
   }
 
-  /** Tests {@link AggregateConstantKeyRule} where both keys are constants but
-   * only one can be removed. */
+  /** Tests {@link AggregateProjectPullUpConstantsRule} where both keys are
+   * constants but only one can be removed. */
   @Test public void testAggregateConstantKeyRule3() {
     final HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(AggregateConstantKeyRule.INSTANCE)
+        .addRuleInstance(AggregateProjectPullUpConstantsRule.INSTANCE2)
         .build();
     final String sql = "select job\n"
         + "from sales.emp\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/904c73da/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 38edf69..c9c9342 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -1795,8 +1795,8 @@ LogicalProject(DEPTNO=[$0], EXPR$1=[$4])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(DEPTNO=[$0], EXPR$1=[$2])
-  LogicalAggregate(group=[{0, 1}], EXPR$1=[MAX($4)])
-    LogicalProject(DEPTNO=[$7], DEPTNO42=[+($7, 42)], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], MGR=[$3])
+  LogicalAggregate(group=[{0, 3}], EXPR$1=[MAX($4)])
+    LogicalProject(DEPTNO=[$7], FOUR=[4], TWO_PLUS_THREE=[+(2, 3)], DEPTNO42=[+($7, 42)], MGR=[$3])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -1818,8 +1818,8 @@ LogicalProject(DEPTNO=[$1], EXPR$1=[$2])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
-  LogicalProject(DEPTNO=[$7], FOUR=[4], MGR=[$3])
+LogicalAggregate(group=[{1}], EXPR$1=[MAX($2)])
+  LogicalProject(FOUR=[4], DEPTNO=[$7], MGR=[$3])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -1841,8 +1841,8 @@ LogicalProject(DEPTNO=[$1], EXPR$1=[$2])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalAggregate(group=[{0}], EXPR$1=[MAX($2)])
-  LogicalProject(DEPTNO=[$7], $f0=[+(42, 24)], MGR=[$3])
+LogicalAggregate(group=[{1}], EXPR$1=[MAX($2)])
+  LogicalProject($f0=[+(42, 24)], DEPTNO=[$7], MGR=[$3])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -4311,7 +4311,7 @@ LogicalProject(JOB=[$1])
             <![CDATA[
 LogicalProject(JOB=[$1])
   LogicalFilter(condition=[>($2, 3)])
-    LogicalProject(SAL=[$0], JOB=['Clerk'], $f1=[$1])
+    LogicalProject(SAL=[$0], JOB=['Clerk'], $f2=[$1])
       LogicalAggregate(group=[{0}], agg#0=[COUNT()])
         LogicalProject(SAL=[$5], JOB=[$2])
           LogicalFilter(condition=[AND(IS NULL($5), =($2, 'Clerk'))])