You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2016/10/18 11:28:37 UTC

[1/6] hive git commit: HIVE-13316: Upgrade to Calcite 1.10 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 4b7f373e5 -> b597ab2a0


http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out b/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
index 71e470b..87d4c70 100644
--- a/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorization_limit.q.out
@@ -434,7 +434,14 @@ STAGE PLANS:
     Fetch Operator
       limit: 0
       Processor Tree:
-        ListSink
+        TableScan
+          alias: alltypesorc
+          Select Operator
+            expressions: ctinyint (type: tinyint), cdouble (type: double)
+            outputColumnNames: _col0, _col1
+            Limit
+              Number of rows: 0
+              ListSink
 
 PREHOOK: query: select ctinyint,cdouble from alltypesorc order by ctinyint limit 0
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/vectorization_limit.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorization_limit.q.out b/ql/src/test/results/clientpositive/vectorization_limit.q.out
index eb2a692..cebbcdf 100644
--- a/ql/src/test/results/clientpositive/vectorization_limit.q.out
+++ b/ql/src/test/results/clientpositive/vectorization_limit.q.out
@@ -417,7 +417,17 @@ STAGE PLANS:
     Fetch Operator
       limit: 0
       Processor Tree:
-        ListSink
+        TableScan
+          alias: alltypesorc
+          Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: ctinyint (type: tinyint), cdouble (type: double)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 0
+              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+              ListSink
 
 PREHOOK: query: select ctinyint,cdouble from alltypesorc order by ctinyint limit 0
 PREHOOK: type: QUERY


[4/6] hive git commit: HIVE-13316: Upgrade to Calcite 1.10 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTSTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTSTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTSTransposeRule.java
index f81c21b..49e4bec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTSTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTSTransposeRule.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
+import org.apache.calcite.adapter.druid.DruidQuery;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
@@ -32,17 +33,28 @@ import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 
 //TODO: Remove this once Calcite FilterProjectTransposeRule can take rule operand
 public class HiveFilterProjectTSTransposeRule extends RelOptRule {
 
+  public final static HiveFilterProjectTSTransposeRule INSTANCE =
+      new HiveFilterProjectTSTransposeRule(
+          Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY, HiveProject.class,
+          HiveRelFactories.HIVE_PROJECT_FACTORY, TableScan.class);
+
+  public final static  HiveFilterProjectTSTransposeRule INSTANCE_DRUID =
+      new HiveFilterProjectTSTransposeRule(
+          Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY, HiveProject.class,
+          HiveRelFactories.HIVE_PROJECT_FACTORY, DruidQuery.class);
+
   private final RelFactories.FilterFactory  filterFactory;
   private final RelFactories.ProjectFactory projectFactory;
 
-  public HiveFilterProjectTSTransposeRule(Class<? extends Filter> filterClass,
+  private HiveFilterProjectTSTransposeRule(Class<? extends Filter> filterClass,
       FilterFactory filterFactory, Class<? extends Project> projectClass,
-      ProjectFactory projectFactory, Class<? extends TableScan> tsClass) {
+      ProjectFactory projectFactory, Class<? extends RelNode> tsClass) {
     super(operand(filterClass, operand(projectClass, operand(tsClass, none()))));
     this.filterFactory = filterFactory;
     this.projectFactory = projectFactory;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTransposeRule.java
index d43c2c6..91d674d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveFilterProjectTransposeRule.java
@@ -27,8 +27,6 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.RelFactories.FilterFactory;
-import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexCall;
@@ -37,6 +35,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexOver;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
@@ -44,25 +43,25 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 public class HiveFilterProjectTransposeRule extends FilterProjectTransposeRule {
 
   public static final HiveFilterProjectTransposeRule INSTANCE_DETERMINISTIC_WINDOWING =
-          new HiveFilterProjectTransposeRule(Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY,
-          HiveProject.class, HiveRelFactories.HIVE_PROJECT_FACTORY, true, true);
+          new HiveFilterProjectTransposeRule(Filter.class, HiveProject.class,
+                  HiveRelFactories.HIVE_BUILDER, true, true);
 
   public static final HiveFilterProjectTransposeRule INSTANCE_DETERMINISTIC =
-          new HiveFilterProjectTransposeRule(Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY,
-          HiveProject.class, HiveRelFactories.HIVE_PROJECT_FACTORY, true, false);
+          new HiveFilterProjectTransposeRule(Filter.class, HiveProject.class,
+                  HiveRelFactories.HIVE_BUILDER, true, false);
 
   public static final HiveFilterProjectTransposeRule INSTANCE =
-          new HiveFilterProjectTransposeRule(Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY,
-          HiveProject.class, HiveRelFactories.HIVE_PROJECT_FACTORY, false, false);
+          new HiveFilterProjectTransposeRule(Filter.class, HiveProject.class,
+                  HiveRelFactories.HIVE_BUILDER, false, false);
 
   private final boolean onlyDeterministic;
 
   private final boolean pushThroughWindowing;
 
   private HiveFilterProjectTransposeRule(Class<? extends Filter> filterClass,
-      FilterFactory filterFactory, Class<? extends Project> projectClass,
-      ProjectFactory projectFactory, boolean onlyDeterministic,boolean pushThroughWindowing) {
-    super(filterClass, filterFactory, projectClass, projectFactory);
+      Class<? extends Project> projectClass, RelBuilderFactory relBuilderFactory,
+      boolean onlyDeterministic,boolean pushThroughWindowing) {
+    super(filterClass, projectClass, false, false, relBuilderFactory);
     this.onlyDeterministic = onlyDeterministic;
     this.pushThroughWindowing = pushThroughWindowing;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
index 7c2a7e5..dccb6a5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePreFilteringRule.java
@@ -39,7 +39,6 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,9 +49,7 @@ import com.google.common.collect.Sets;
 
 public class HivePreFilteringRule extends RelOptRule {
 
-  protected static final Logger            LOG        = LoggerFactory
-                                                          .getLogger(HivePreFilteringRule.class
-                                                              .getName());
+  protected static final Logger LOG = LoggerFactory.getLogger(HivePreFilteringRule.class);
 
   private static final Set<SqlKind>        COMPARISON = EnumSet.of(SqlKind.EQUALS,
                                                           SqlKind.GREATER_THAN_OR_EQUAL,
@@ -209,7 +206,7 @@ public class HivePreFilteringRule extends RelOptRule {
     for (int i = 0; i < operands.size(); i++) {
       final RexNode operand = operands.get(i);
 
-      final RexNode operandCNF = HiveRexUtil.toCnf(rexBuilder, maxCNFNodeCount, operand);
+      final RexNode operandCNF = RexUtil.toCnf(rexBuilder, maxCNFNodeCount, operand);
       final List<RexNode> conjunctions = RelOptUtil.conjunctions(operandCNF);
 
       Set<String> refsInCurrentOperand = Sets.newHashSet();

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
index 2fc68ae..a1b5aeb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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.
@@ -16,59 +16,28 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
 
-import org.apache.calcite.plan.RelOptPlanner;
 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.Filter;
-import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.rules.ValuesReduceRule;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rel.rules.ReduceExpressionsRule;
 import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexCorrelVariable;
-import org.apache.calcite.rex.RexDynamicParam;
-import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexOver;
-import org.apache.calcite.rex.RexRangeRef;
-import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.fun.SqlRowOperator;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Stacks;
-import org.apache.calcite.util.Util;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexUtil;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexUtil.ExprSimplifier;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
 /**
@@ -82,51 +51,55 @@ import com.google.common.collect.Lists;
  * is the same as the type of the resulting cast expression
  * </ul>
  */
-public abstract class HiveReduceExpressionsRule extends RelOptRule {
+public abstract class HiveReduceExpressionsRule extends ReduceExpressionsRule {
 
   protected static final Logger LOG = LoggerFactory.getLogger(HiveReduceExpressionsRule.class);
 
   //~ Static fields/initializers ---------------------------------------------
 
   /**
-   * Regular expression that matches the description of all instances of this
-   * rule and {@link ValuesReduceRule} also. Use
-   * it to prevent the planner from invoking these rules.
-   */
-  public static final Pattern EXCLUSION_PATTERN =
-      Pattern.compile("Reduce(Expressions|Values)Rule.*");
-
-  /**
    * Singleton rule that reduces constants inside a
-   * {@link org.apache.calcite.rel.logical.HiveFilter}.
+   * {@link org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter}.
    */
-  public static final HiveReduceExpressionsRule FILTER_INSTANCE =
-      new FilterReduceExpressionsRule(HiveFilter.class, HiveRelFactories.HIVE_BUILDER);
+  public static final ReduceExpressionsRule FILTER_INSTANCE =
+          new FilterReduceExpressionsRule(HiveFilter.class, HiveRelFactories.HIVE_BUILDER);
 
   /**
    * Singleton rule that reduces constants inside a
-   * {@link org.apache.calcite.rel.logical.HiveProject}.
+   * {@link org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject}.
    */
-  public static final HiveReduceExpressionsRule PROJECT_INSTANCE =
+  public static final ReduceExpressionsRule PROJECT_INSTANCE =
       new ProjectReduceExpressionsRule(HiveProject.class, HiveRelFactories.HIVE_BUILDER);
 
   /**
    * Singleton rule that reduces constants inside a
-   * {@link org.apache.calcite.rel.core.HiveJoin}.
+   * {@link org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin}.
    */
-  public static final HiveReduceExpressionsRule JOIN_INSTANCE =
+  public static final ReduceExpressionsRule JOIN_INSTANCE =
       new JoinReduceExpressionsRule(HiveJoin.class, HiveRelFactories.HIVE_BUILDER);
 
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Creates a HiveReduceExpressionsRule.
+   *
+   * @param clazz class of rels to which this rule should apply
+   */
+  protected HiveReduceExpressionsRule(Class<? extends RelNode> clazz,
+      RelBuilderFactory relBuilderFactory, String desc) {
+    super(clazz, relBuilderFactory, desc);
+  }
+
   /**
    * Rule that reduces constants inside a {@link org.apache.calcite.rel.core.Filter}.
    * If the condition is a constant, the filter is removed (if TRUE) or replaced with
    * an empty {@link org.apache.calcite.rel.core.Values} (if FALSE or NULL).
    */
-  public static class FilterReduceExpressionsRule extends HiveReduceExpressionsRule {
+  public static class FilterReduceExpressionsRule extends ReduceExpressionsRule {
 
     public FilterReduceExpressionsRule(Class<? extends Filter> filterClass,
         RelBuilderFactory relBuilderFactory) {
-      super(filterClass, relBuilderFactory, "HiveReduceExpressionsRule(Filter)");
+      super(filterClass, relBuilderFactory, "ReduceExpressionsRule(Filter)");
     }
 
     @Override public void onMatch(RelOptRuleCall call) {
@@ -135,8 +108,9 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
           Lists.newArrayList(filter.getCondition());
       RexNode newConditionExp;
       boolean reduced;
+      final RelMetadataQuery mq = RelMetadataQuery.instance();
       final RelOptPredicateList predicates =
-          RelMetadataQuery.instance().getPulledUpPredicates(filter.getInput());
+          mq.getPulledUpPredicates(filter.getInput());
       if (reduceExpressions(filter, expList, predicates, true)) {
         assert expList.size() == 1;
         newConditionExp = expList.get(0);
@@ -154,795 +128,95 @@ public abstract class HiveReduceExpressionsRule extends RelOptRule {
       // predicate to see if it was already a constant,
       // in which case we don't need any runtime decision
       // about filtering.
-      // TODO: support LogicalValues
       if (newConditionExp.isAlwaysTrue()) {
         call.transformTo(
             filter.getInput());
       } else if (reduced) {
+        if (RexUtil.isNullabilityCast(filter.getCluster().getTypeFactory(),
+            newConditionExp)) {
+          newConditionExp = ((RexCall) newConditionExp).getOperands().get(0);
+        }
         call.transformTo(call.builder().
             push(filter.getInput()).filter(newConditionExp).build());
       } else {
+        if (newConditionExp instanceof RexCall) {
+          RexCall rexCall = (RexCall) newConditionExp;
+          boolean reverse = rexCall.getKind() == SqlKind.NOT;
+          if (reverse) {
+            if (!(rexCall.getOperands().get(0) instanceof RexCall)) {
+              // If child is not a RexCall instance, we can bail out
+              return;
+            }
+            rexCall = (RexCall) rexCall.getOperands().get(0);
+          }
+          reduceNotNullableFilter(call, filter, rexCall, reverse);
+        }
         return;
       }
 
       // New plan is absolutely better than old plan.
       call.getPlanner().setImportance(filter, 0.0);
     }
-  }
-
-  /**
-   * Rule that reduces constants inside a {@link org.apache.calcite.rel.core.Project}.
-   */
-  public static class ProjectReduceExpressionsRule extends HiveReduceExpressionsRule {
-
-    public ProjectReduceExpressionsRule(Class<? extends Project> projectClass,
-        RelBuilderFactory relBuilderFactory) {
-      super(projectClass, relBuilderFactory, "HiveReduceExpressionsRule(Project)");
-    }
-
-    @Override public void onMatch(RelOptRuleCall call) {
-      Project project = call.rel(0);
-      final RelOptPredicateList predicates =
-          RelMetadataQuery.instance().getPulledUpPredicates(project.getInput());
-      final List<RexNode> expList =
-          Lists.newArrayList(project.getProjects());
-      if (reduceExpressions(project, expList, predicates)) {
-        RelNode newProject = call.builder().push(project.getInput())
-            .project(expList, project.getRowType().getFieldNames()).build();
-        call.transformTo(newProject);
 
-        // New plan is absolutely better than old plan.
-        call.getPlanner().setImportance(project, 0.0);
-      }
-    }
-  }
-
-  /**
-   * Rule that reduces constants inside a {@link org.apache.calcite.rel.core.HiveJoin}.
-   */
-  public static class JoinReduceExpressionsRule extends HiveReduceExpressionsRule {
-
-    public JoinReduceExpressionsRule(Class<? extends HiveJoin> joinClass,
-        RelBuilderFactory relBuilderFactory) {
-      super(joinClass, relBuilderFactory, "HiveReduceExpressionsRule(HiveJoin)");
-    }
-
-    @Override public void onMatch(RelOptRuleCall call) {
-      final HiveJoin join = call.rel(0);
-      final List<RexNode> expList = Lists.newArrayList(join.getCondition());
-      final int fieldCount = join.getLeft().getRowType().getFieldCount();
-      RelMetadataQuery mq = RelMetadataQuery.instance();
-      final RelOptPredicateList leftPredicates =
-          mq.getPulledUpPredicates(join.getLeft());
-      final RelOptPredicateList rightPredicates =
-          mq.getPulledUpPredicates(join.getRight());
-      final RelOptPredicateList predicates =
-          leftPredicates.union(rightPredicates.shift(fieldCount));
-      if (!reduceExpressions(join, expList, predicates, true)) {
-        return;
-      }
-      call.transformTo(
-          join.copy(
-              join.getTraitSet(),
-              expList.get(0),
-              join.getLeft(),
-              join.getRight(),
-              join.getJoinType(),
-              join.isSemiJoinDone()));
-
-      // New plan is absolutely better than old plan.
-      call.getPlanner().setImportance(join, 0.0);
-    }
-  }
-
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Creates a HiveReduceExpressionsRule.
-   *
-   * @param clazz class of rels to which this rule should apply
-   */
-  protected HiveReduceExpressionsRule(Class<? extends RelNode> clazz,
-      RelBuilderFactory relBuilderFactory, String desc) {
-    super(operand(clazz, any()), relBuilderFactory, desc);
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  /**
-   * 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
-   * @return whether reduction found something to change, and succeeded
-   */
-  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
-    ExprSimplifier simplifier = new ExprSimplifier(rexBuilder, unknownAsFalse);
-    List<RexNode> expList2 = Lists.newArrayList(expList);
-    simplifier.mutate(expList2);
-    boolean simplified = false;
-    for (int i = 0; i < expList.size(); i++) {
-      if (!expList2.get(i).toString().equals(expList.get(i).toString())) {
-        expList.remove(i);
-        expList.add(i, expList2.get(i));
-        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.
-    new CaseShuttle().mutate(expList);
-
-    // Find reducible expressions.
-    final List<RexNode> constExps = Lists.newArrayList();
-    List<Boolean> addCasts = Lists.newArrayList();
-    final List<RexNode> removableCasts = Lists.newArrayList();
-    final ImmutableMap<RexNode, RexNode> constants =
-        predicateConstants(RexNode.class, rexBuilder, predicates);
-    findReducibleExps(rel.getCluster().getTypeFactory(), expList, constants,
-        constExps, addCasts, removableCasts);
-    if (constExps.isEmpty() && removableCasts.isEmpty()) {
-      return false;
-    }
-
-    // Remove redundant casts before reducing constant expressions.
-    // If the argument to the redundant cast is a reducible constant,
-    // reducing that argument to a constant first will result in not being
-    // able to locate the original cast expression.
-    if (!removableCasts.isEmpty()) {
-      final List<RexNode> reducedExprs = Lists.newArrayList();
-      for (RexNode exp : removableCasts) {
-        RexCall call = (RexCall) exp;
-        reducedExprs.add(call.getOperands().get(0));
-      }
-      RexReplacer replacer =
-          new RexReplacer(
-              rexBuilder,
-              removableCasts,
-              reducedExprs,
-              Collections.nCopies(removableCasts.size(), false));
-      replacer.mutate(expList);
-    }
-
-    if (constExps.isEmpty()) {
-      return true;
-    }
-
-    final List<RexNode> constExps2 = Lists.newArrayList(constExps);
-    if (!constants.isEmpty()) {
-      //noinspection unchecked
-      final List<Map.Entry<RexNode, RexNode>> pairs =
-          (List<Map.Entry<RexNode, RexNode>>) (List)
-              Lists.newArrayList(constants.entrySet());
-      RexReplacer replacer =
-          new RexReplacer(
-              rexBuilder,
-              Pair.left(pairs),
-              Pair.right(pairs),
-              Collections.nCopies(pairs.size(), false));
-      replacer.mutate(constExps2);
-    }
-
-    // Compute the values they reduce to.
-    RelOptPlanner.Executor executor =
-        rel.getCluster().getPlanner().getExecutor();
-    if (executor == null) {
-      // Cannot reduce expressions: caller has not set an executor in their
-      // environment. Caller should execute something like the following before
-      // invoking the planner:
-      //
-      // final RexExecutorImpl executor =
-      //   new RexExecutorImpl(Schemas.createDataContext(null));
-      // rootRel.getCluster().getPlanner().setExecutor(executor);
-      return false;
-    }
-
-    final List<RexNode> reducedValues = Lists.newArrayList();
-    executor.reduce(rexBuilder, constExps2, reducedValues);
-
-    // Use RexNode.digest to judge whether each newly generated RexNode
-    // is equivalent to the original one.
-    if (Lists.transform(constExps, HiveCalciteUtil.REX_STR_FN).equals(
-            Lists.transform(reducedValues, HiveCalciteUtil.REX_STR_FN))) {
-      return false;
-    }
-
-    // For Project, we have to be sure to preserve the result
-    // types, so always cast regardless of the expression type.
-    // For other RelNodes like Filter, in general, this isn't necessary,
-    // and the presence of casts could hinder other rules such as sarg
-    // analysis, which require bare literals.  But there are special cases,
-    // like when the expression is a UDR argument, that need to be
-    // handled as special cases.
-    if (rel instanceof Project) {
-      addCasts = Collections.nCopies(reducedValues.size(), true);
-    }
-
-    RexReplacer replacer =
-        new RexReplacer(
-            rexBuilder,
-            constExps,
-            reducedValues,
-            addCasts);
-    replacer.mutate(expList);
-    return true;
-  }
-
-  /**
-   * Locates expressions that can be reduced to literals or converted to
-   * expressions with redundant casts removed.
-   *
-   * @param typeFactory    Type factory
-   * @param exps           list of candidate expressions to be examined for
-   *                       reduction
-   * @param constants      List of expressions known to be constant
-   * @param constExps      returns the list of expressions that can be constant
-   *                       reduced
-   * @param addCasts       indicator for each expression that can be constant
-   *                       reduced, whether a cast of the resulting reduced
-   *                       expression is potentially necessary
-   * @param removableCasts returns the list of cast expressions where the cast
-   */
-  protected static void findReducibleExps(RelDataTypeFactory typeFactory,
-      List<RexNode> exps, ImmutableMap<RexNode, RexNode> constants,
-      List<RexNode> constExps, List<Boolean> addCasts,
-      List<RexNode> removableCasts) {
-    ReducibleExprLocator gardener =
-        new ReducibleExprLocator(typeFactory, constants, constExps,
-            addCasts, removableCasts);
-    for (RexNode exp : exps) {
-      gardener.analyze(exp);
-    }
-    assert constExps.size() == addCasts.size();
-  }
-
-  /** 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
-   */
-  public 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
-    // matter which we take, so the latter will replace the former.
-    // The basic idea is to find all the pairs of RexNode = RexLiteral
-    // (1) If 'predicates' contain a non-EQUALS, we bail out.
-    // (2) It is OK if a RexNode is equal to the same RexLiteral several times,
-    // (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, C> map = new HashMap<>();
-    final Set<RexNode> excludeSet = new HashSet<>();
-    for (RexNode predicate : predicates.pulledUpPredicates) {
-      gatherConstraints(clazz, predicate, map, excludeSet, rexBuilder);
-    }
-    final ImmutableMap.Builder<RexNode, C> builder =
-        ImmutableMap.builder();
-    for (Map.Entry<RexNode, C> entry : map.entrySet()) {
-      RexNode rexNode = entry.getKey();
-      if (!overlap(rexNode, excludeSet)) {
-        builder.put(rexNode, entry.getValue());
-      }
-    }
-    return builder.build();
-  }
-
-  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
-            && predicate.getKind() != SqlKind.IS_NULL) {
-      decompose(excludeSet, predicate);
-      return;
-    }
-    final List<RexNode> operands = ((RexCall) predicate).getOperands();
-    if (operands.size() != 2 && predicate.getKind() == SqlKind.EQUALS) {
-      decompose(excludeSet, predicate);
-      return;
-    }
-    // if it reaches here, we have rexNode equals rexNode
-    final RexNode left;
-    final RexNode right;
-    if (predicate.getKind() == SqlKind.EQUALS) {
-      left = operands.get(0);
-      right = operands.get(1);
-    } else {
-      left = operands.get(0);
-      right = rexBuilder.makeNullLiteral(left.getType().getSqlTypeName());
-    }
-    // note that literals are immutable too and they can only be compared through
-    // values.
-    gatherConstraint(clazz, left, right, map, excludeSet, rexBuilder);
-    gatherConstraint(clazz, right, left, map, excludeSet, rexBuilder);
-  }
-
-  /** Returns whether a value of {@code type2} can be assigned to a variable
-   * of {@code type1}.
-   *
-   * <p>For example:
-   * <ul>
-   *   <li>{@code canAssignFrom(BIGINT, TINYINT)} returns {@code true}</li>
-   *   <li>{@code canAssignFrom(TINYINT, BIGINT)} returns {@code false}</li>
-   *   <li>{@code canAssignFrom(BIGINT, VARCHAR)} returns {@code false}</li>
-   * </ul>
-   */
-  private static boolean canAssignFrom(RelDataType type1, RelDataType type2) {
-    final SqlTypeName name1 = type1.getSqlTypeName();
-    final SqlTypeName name2 = type2.getSqlTypeName();
-    if (name1.getFamily() == name2.getFamily()) {
-      switch (name1.getFamily()) {
-      case NUMERIC:
-        return name1.compareTo(name2) >= 0;
+    /**
+     * For static schema systems, a filter that is always false or null can be
+     * replaced by a values operator that produces no rows, as the schema
+     * information can just be taken from the input Rel. In dynamic schema
+     * environments, the filter might have an unknown input type, in these cases
+     * they must define a system specific alternative to a Values operator, such
+     * as inserting a limit 0 instead of a filter on top of the original input.
+     *
+     * <p>The default implementation of this method is to call
+     * {@link RelBuilder#empty}, which for the static schema will be optimized
+     * to an empty
+     * {@link org.apache.calcite.rel.core.Values}.
+     *
+     * @param input rel to replace, assumes caller has already determined
+     *              equivalence to Values operation for 0 records or a
+     *              false filter.
+     * @return equivalent but less expensive replacement rel
+     */
+    protected RelNode createEmptyRelOrEquivalent(RelOptRuleCall call, Filter input) {
+      return call.builder().push(input).empty().build();
+    }
+
+    private void reduceNotNullableFilter(
+        RelOptRuleCall call,
+        Filter filter,
+        RexCall rexCall,
+        boolean reverse) {
+      // If the expression is a IS [NOT] NULL on a non-nullable
+      // column, then we can either remove the filter or replace
+      // it with an Empty.
+      boolean alwaysTrue;
+      switch (rexCall.getKind()) {
+      case IS_NULL:
+      case IS_UNKNOWN:
+        alwaysTrue = false;
+        break;
+      case IS_NOT_NULL:
+        alwaysTrue = true;
+        break;
       default:
-        return true;
-      }
-    }
-    return false;
-  }
-
-  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 C existedValue = map.get(left);
-    if (existedValue == null) {
-      switch (left.getKind()) {
-      case CAST:
-        // Convert "CAST(c) = literal" to "c = literal", as long as it is a
-        // widening cast.
-        final RexNode operand = ((RexCall) left).getOperands().get(0);
-        if (canAssignFrom(left.getType(), operand.getType())) {
-          final RexNode castRight =
-              rexBuilder.makeCast(operand.getType(), constant);
-          if (castRight instanceof RexLiteral) {
-            left = operand;
-            constant = clazz.cast(castRight);
-          }
-        }
-      }
-      map.put(left, constant);
-    } else {
-      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);
-      }
-    }
-  }
-
-  private static boolean overlap(RexNode rexNode, Set<RexNode> set) {
-    if (rexNode instanceof RexCall) {
-      for (RexNode r : ((RexCall) rexNode).getOperands()) {
-        if (overlap(r, set)) {
-          return true;
-        }
-      }
-      return false;
-    } else {
-      return set.contains(rexNode);
-    }
-  }
-
-  /** Tries to decompose the RexNode which is a RexCall into non-literal
-   * RexNodes. */
-  private static void decompose(Set<RexNode> set, RexNode rexNode) {
-    if (rexNode instanceof RexCall) {
-      for (RexNode r : ((RexCall) rexNode).getOperands()) {
-        decompose(set, r);
-      }
-    } else if (!(rexNode instanceof RexLiteral)) {
-      set.add(rexNode);
-    }
-  }
-
-  /** Pushes predicates into a CASE.
-   *
-   * <p>We have a loose definition of 'predicate': any boolean expression will
-   * do, except CASE. For example '(CASE ...) = 5' or '(CASE ...) IS NULL'.
-   */
-  protected static RexCall pushPredicateIntoCase(RexCall call) {
-    if (call.getType().getSqlTypeName() != SqlTypeName.BOOLEAN) {
-      return call;
-    }
-    switch (call.getKind()) {
-    case CASE:
-    case AND:
-    case OR:
-      return call; // don't push CASE into CASE!
-    }
-    int caseOrdinal = -1;
-    final List<RexNode> operands = call.getOperands();
-    for (int i = 0; i < operands.size(); i++) {
-      RexNode operand = operands.get(i);
-      switch (operand.getKind()) {
-      case CASE:
-        caseOrdinal = i;
-      }
-    }
-    if (caseOrdinal < 0) {
-      return call;
-    }
-    // Convert
-    //   f(CASE WHEN p1 THEN v1 ... END, arg)
-    // to
-    //   CASE WHEN p1 THEN f(v1, arg) ... END
-    final RexCall case_ = (RexCall) operands.get(caseOrdinal);
-    final List<RexNode> nodes = new ArrayList<>();
-    for (int i = 0; i < case_.getOperands().size(); i++) {
-      RexNode node = case_.getOperands().get(i);
-      if (!RexUtil.isCasePredicate(case_, i)) {
-        node = substitute(call, caseOrdinal, node);
-      }
-      nodes.add(node);
-    }
-    return case_.clone(call.getType(), nodes);
-  }
-
-  /** Converts op(arg0, ..., argOrdinal, ..., argN) to op(arg0,..., node, ..., argN). */
-  protected static RexNode substitute(RexCall call, int ordinal, RexNode node) {
-    final List<RexNode> newOperands = Lists.newArrayList(call.getOperands());
-    newOperands.set(ordinal, node);
-    return call.clone(call.getType(), newOperands);
-  }
-
-  //~ Inner Classes ----------------------------------------------------------
-
-  /**
-   * Replaces expressions with their reductions. Note that we only have to
-   * look for RexCall, since nothing else is reducible in the first place.
-   */
-  protected static class RexReplacer extends RexShuttle {
-    private final RexBuilder rexBuilder;
-    private final List<RexNode> reducibleExps;
-    private final List<RexNode> reducedValues;
-    private final List<Boolean> addCasts;
-
-    RexReplacer(
-        RexBuilder rexBuilder,
-        List<RexNode> reducibleExps,
-        List<RexNode> reducedValues,
-        List<Boolean> addCasts) {
-      this.rexBuilder = rexBuilder;
-      this.reducibleExps = reducibleExps;
-      this.reducedValues = reducedValues;
-      this.addCasts = addCasts;
-    }
-
-    @Override public RexNode visitInputRef(RexInputRef inputRef) {
-      RexNode node = visit(inputRef);
-      if (node == null) {
-        return super.visitInputRef(inputRef);
-      }
-      return node;
-    }
-
-    @Override public RexNode visitCall(RexCall call) {
-      RexNode node = visit(call);
-      if (node != null) {
-        return node;
-      }
-      node = super.visitCall(call);
-      if (node != call) {
-        node = HiveRexUtil.simplify(rexBuilder, node);
-      }
-      return node;
-    }
-
-    private RexNode visit(final RexNode call) {
-      int i = reducibleExps.indexOf(call);
-      if (i == -1) {
-        return null;
-      }
-      RexNode replacement = reducedValues.get(i);
-      if (addCasts.get(i)
-          && (replacement.getType() != call.getType())) {
-        // Handle change from nullable to NOT NULL by claiming
-        // that the result is still nullable, even though
-        // we know it isn't.
-        //
-        // Also, we cannot reduce CAST('abc' AS VARCHAR(4)) to 'abc'.
-        // If we make 'abc' of type VARCHAR(4), we may later encounter
-        // the same expression in a Project's digest where it has
-        // type VARCHAR(3), and that's wrong.
-        replacement = rexBuilder.makeAbstractCast(call.getType(), replacement);
-      }
-      return replacement;
-    }
-  }
-
-  /**
-   * Helper class used to locate expressions that either can be reduced to
-   * literals or contain redundant casts.
-   */
-  protected static class ReducibleExprLocator extends RexVisitorImpl<Void> {
-    /** Whether an expression is constant, and if so, whether it can be
-     * reduced to a simpler constant. */
-    enum Constancy {
-      NON_CONSTANT, REDUCIBLE_CONSTANT, IRREDUCIBLE_CONSTANT
-    }
-
-    private final RelDataTypeFactory typeFactory;
-
-    private final List<Constancy> stack;
-
-    private final ImmutableMap<RexNode, RexNode> constants;
-
-    private final List<RexNode> constExprs;
-
-    private final List<Boolean> addCasts;
-
-    private final List<RexNode> removableCasts;
-
-    private final List<SqlOperator> parentCallTypeStack;
-
-    ReducibleExprLocator(RelDataTypeFactory typeFactory,
-        ImmutableMap<RexNode, RexNode> constants, List<RexNode> constExprs,
-        List<Boolean> addCasts, List<RexNode> removableCasts) {
-      // go deep
-      super(true);
-      this.typeFactory = typeFactory;
-      this.constants = constants;
-      this.constExprs = constExprs;
-      this.addCasts = addCasts;
-      this.removableCasts = removableCasts;
-      this.stack = Lists.newArrayList();
-      this.parentCallTypeStack = Lists.newArrayList();
-    }
-
-    public void analyze(RexNode exp) {
-      assert stack.isEmpty();
-
-      exp.accept(this);
-
-      // Deal with top of stack
-      assert stack.size() == 1;
-      assert parentCallTypeStack.isEmpty();
-      Constancy rootConstancy = stack.get(0);
-      if (rootConstancy == Constancy.REDUCIBLE_CONSTANT) {
-        // The entire subtree was constant, so add it to the result.
-        addResult(exp);
-      }
-      stack.clear();
-    }
-
-    private Void pushVariable() {
-      stack.add(Constancy.NON_CONSTANT);
-      return null;
-    }
-
-    private void addResult(RexNode exp) {
-      // Cast of literal can't be reduced, so skip those (otherwise we'd
-      // go into an infinite loop as we add them back).
-      if (exp.getKind() == SqlKind.CAST) {
-        RexCall cast = (RexCall) exp;
-        RexNode operand = cast.getOperands().get(0);
-        if (operand instanceof RexLiteral) {
-          return;
-        }
-      }
-      constExprs.add(exp);
-
-      // In the case where the expression corresponds to a UDR argument,
-      // we need to preserve casts.  Note that this only applies to
-      // the topmost argument, not expressions nested within the UDR
-      // call.
-      //
-      // REVIEW zfong 6/13/08 - Are there other expressions where we
-      // also need to preserve casts?
-      if (parentCallTypeStack.isEmpty()) {
-        addCasts.add(false);
-      } else {
-        addCasts.add(isUdf(Stacks.peek(parentCallTypeStack)));
-      }
-    }
-
-    private Boolean isUdf(SqlOperator operator) {
-      // return operator instanceof UserDefinedRoutine
-      return false;
-    }
-
-    public Void visitInputRef(RexInputRef inputRef) {
-      if (constants.containsKey(inputRef)) {
-        stack.add(Constancy.REDUCIBLE_CONSTANT);
-        return null;
-      }
-      return pushVariable();
-    }
-
-    public Void visitLiteral(RexLiteral literal) {
-      stack.add(Constancy.IRREDUCIBLE_CONSTANT);
-      return null;
-    }
-
-    public Void visitOver(RexOver over) {
-      // assume non-constant (running SUM(1) looks constant but isn't)
-      analyzeCall(over, Constancy.NON_CONSTANT);
-      return null;
-    }
-
-    public Void visitCorrelVariable(RexCorrelVariable correlVariable) {
-      return pushVariable();
-    }
-
-    public Void visitCall(RexCall call) {
-      // assume REDUCIBLE_CONSTANT until proven otherwise
-      analyzeCall(call, Constancy.REDUCIBLE_CONSTANT);
-      return null;
-    }
-
-    private void analyzeCall(RexCall call, Constancy callConstancy) {
-      Stacks.push(parentCallTypeStack, call.getOperator());
-
-      // visit operands, pushing their states onto stack
-      super.visitCall(call);
-
-      // look for NON_CONSTANT operands
-      int operandCount = call.getOperands().size();
-      List<Constancy> operandStack = Util.last(stack, operandCount);
-      for (Constancy operandConstancy : operandStack) {
-        if (operandConstancy == Constancy.NON_CONSTANT) {
-          callConstancy = Constancy.NON_CONSTANT;
-        }
-      }
-
-      // Even if all operands are constant, the call itself may
-      // be non-deterministic.
-      if (!call.getOperator().isDeterministic()) {
-        callConstancy = Constancy.NON_CONSTANT;
-      } else if (call.getOperator().isDynamicFunction()) {
-        // We can reduce the call to a constant, but we can't
-        // cache the plan if the function is dynamic.
-        // For now, treat it same as non-deterministic.
-        callConstancy = Constancy.NON_CONSTANT;
-      }
-
-      // Row operator itself can't be reduced to a literal, but if
-      // the operands are constants, we still want to reduce those
-      if ((callConstancy == Constancy.REDUCIBLE_CONSTANT)
-          && (call.getOperator() instanceof SqlRowOperator)) {
-        callConstancy = Constancy.NON_CONSTANT;
+        return;
       }
-
-      if (callConstancy == Constancy.NON_CONSTANT) {
-        // any REDUCIBLE_CONSTANT children are now known to be maximal
-        // reducible subtrees, so they can be added to the result
-        // list
-        for (int iOperand = 0; iOperand < operandCount; ++iOperand) {
-          Constancy constancy = operandStack.get(iOperand);
-          if (constancy == Constancy.REDUCIBLE_CONSTANT) {
-            addResult(call.getOperands().get(iOperand));
+      if (reverse) {
+        alwaysTrue = !alwaysTrue;
+      }
+      RexNode operand = rexCall.getOperands().get(0);
+      if (operand instanceof RexInputRef) {
+        RexInputRef inputRef = (RexInputRef) operand;
+        if (!inputRef.getType().isNullable()) {
+          if (alwaysTrue) {
+            call.transformTo(filter.getInput());
+          } else {
+            call.transformTo(createEmptyRelOrEquivalent(call, filter));
           }
         }
-
-        // if this cast expression can't be reduced to a literal,
-        // then see if we can remove the cast
-        if (call.getOperator() == SqlStdOperatorTable.CAST) {
-          reduceCasts(call);
-        }
       }
-
-      // pop operands off of the stack
-      operandStack.clear();
-
-      // pop this parent call operator off the stack
-      Stacks.pop(parentCallTypeStack, call.getOperator());
-
-      // push constancy result for this call onto stack
-      stack.add(callConstancy);
-    }
-
-    private void reduceCasts(RexCall outerCast) {
-      List<RexNode> operands = outerCast.getOperands();
-      if (operands.size() != 1) {
-        return;
-      }
-      RelDataType outerCastType = outerCast.getType();
-      RelDataType operandType = operands.get(0).getType();
-      if (operandType.equals(outerCastType)) {
-        removableCasts.add(outerCast);
-        return;
-      }
-
-      // See if the reduction
-      // CAST((CAST x AS type) AS type NOT NULL)
-      // -> CAST(x AS type NOT NULL)
-      // applies.  TODO jvs 15-Dec-2008:  consider
-      // similar cases for precision changes.
-      if (!(operands.get(0) instanceof RexCall)) {
-        return;
-      }
-      RexCall innerCast = (RexCall) operands.get(0);
-      if (innerCast.getOperator() != SqlStdOperatorTable.CAST) {
-        return;
-      }
-      if (innerCast.getOperands().size() != 1) {
-        return;
-      }
-      RelDataType outerTypeNullable =
-          typeFactory.createTypeWithNullability(outerCastType, true);
-      RelDataType innerTypeNullable =
-          typeFactory.createTypeWithNullability(operandType, true);
-      if (outerTypeNullable != innerTypeNullable) {
-        return;
-      }
-      if (operandType.isNullable()) {
-        removableCasts.add(innerCast);
-      }
-    }
-
-    public Void visitDynamicParam(RexDynamicParam dynamicParam) {
-      return pushVariable();
-    }
-
-    public Void visitRangeRef(RexRangeRef rangeRef) {
-      return pushVariable();
-    }
-
-    public Void visitFieldAccess(RexFieldAccess fieldAccess) {
-      return pushVariable();
     }
   }
 
-  /** Shuttle that pushes predicates into a CASE. */
-  protected static class CaseShuttle extends RexShuttle {
-    @Override public RexNode visitCall(RexCall call) {
-      for (;;) {
-        call = (RexCall) super.visitCall(call);
-        final RexCall old = call;
-        call = pushPredicateIntoCase(call);
-        if (call == old) {
-          return call;
-        }
-      }
-    }
-  }
 }
 
 // End HiveReduceExpressionsRule.java

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsWithStatsRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsWithStatsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsWithStatsRule.java
index ec488fe..0644f0c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsWithStatsRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsWithStatsRule.java
@@ -38,7 +38,6 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.Pair;
 import org.apache.hadoop.hive.common.StatsSetupConst;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIn;
 import org.apache.hadoop.hive.ql.plan.ColStatistics;
@@ -131,7 +130,7 @@ public class HiveReduceExpressionsWithStatsRule extends RelOptRule {
             && call.operands.get(0) instanceof RexLiteral) {
           ref = (RexInputRef) call.operands.get(1);
           literal = (RexLiteral) call.operands.get(0);
-          kind = HiveRexUtil.invert(call.getOperator().getKind());
+          kind = call.getOperator().getKind().reverse();
         }
 
         // Found an expression that we can try to reduce
@@ -252,7 +251,7 @@ public class HiveReduceExpressionsWithStatsRule extends RelOptRule {
       // If we did not reduce, check the children nodes
       RexNode node = super.visitCall(call);
       if (node != call) {
-        node = HiveRexUtil.simplify(rexBuilder, node);
+        node = RexUtil.simplify(rexBuilder, node);
       }
       return node;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
index b0cb8df..458fee7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.LinkedHashSet;
@@ -24,28 +25,22 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.calcite.adapter.druid.DruidQuery;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexCorrelVariable;
 import org.apache.calcite.rex.RexFieldAccess;
-import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPermuteInputsShuttle;
-import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.CorrelationReferenceFinder;
@@ -62,7 +57,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.parse.ColumnAccessInfo;
 
 import com.google.common.collect.ImmutableList;
@@ -72,11 +66,9 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
 
   protected static final Log LOG = LogFactory.getLog(HiveRelFieldTrimmer.class);
 
-  private RelBuilder relBuilder;
-
   private ColumnAccessInfo columnAccessInfo;
-
   private Map<HiveProject, Table> viewProjectToTableSchema;
+  private final RelBuilder relBuilder;
 
   public HiveRelFieldTrimmer(SqlValidator validator, RelBuilder relBuilder) {
     super(validator, relBuilder);
@@ -86,9 +78,9 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
   public HiveRelFieldTrimmer(SqlValidator validator, RelBuilder relBuilder,
       ColumnAccessInfo columnAccessInfo, Map<HiveProject, Table> viewToTableSchema) {
     super(validator, relBuilder);
-    this.relBuilder = relBuilder;
     this.columnAccessInfo = columnAccessInfo;
     this.viewProjectToTableSchema = viewToTableSchema;
+    this.relBuilder = relBuilder;
   }
 
   /**
@@ -193,186 +185,73 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
 
   /**
    * Variant of {@link #trimFields(RelNode, ImmutableBitSet, Set)} for
-   * {@link org.apache.calcite.rel.core.Sort}.
+   * {@link org.apache.calcite.adapter.druid.DruidQuery}.
    */
-  public TrimResult trimFields(
-      HiveSortLimit sort,
-      ImmutableBitSet fieldsUsed,
+  public TrimResult trimFields(DruidQuery dq, ImmutableBitSet fieldsUsed,
       Set<RelDataTypeField> extraFields) {
-    final RelDataType rowType = sort.getRowType();
-    final int fieldCount = rowType.getFieldCount();
-    final RelCollation collation = sort.getCollation();
-    final RelNode input = sort.getInput();
-    RelOptCluster cluster = sort.getCluster();
-
-    // We use the fields used by the consumer, plus any fields used as sort
-    // keys.
-    final ImmutableBitSet.Builder inputFieldsUsed =
-        ImmutableBitSet.builder(fieldsUsed);
-    for (RelFieldCollation field : collation.getFieldCollations()) {
-      inputFieldsUsed.set(field.getFieldIndex());
+    final int fieldCount = dq.getRowType().getFieldCount();
+    if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))
+        && extraFields.isEmpty()) {
+      // if there is nothing to project or if we are projecting everything
+      // then no need to introduce another RelNode
+      return trimFields(
+          (RelNode) dq, fieldsUsed, extraFields);
     }
-
-    // Create input with trimmed columns.
-    final Set<RelDataTypeField> inputExtraFields = Collections.emptySet();
-    TrimResult trimResult =
-        trimChild(sort, input, inputFieldsUsed.build(), inputExtraFields);
-    RelNode newInput = trimResult.left;
-    final Mapping inputMapping = trimResult.right;
-
-    // If the input is unchanged, and we need to project all columns,
-    // there's nothing we can do.
-    if (newInput == input
-        && inputMapping.isIdentity()
-        && fieldsUsed.cardinality() == fieldCount) {
-      return result(sort, Mappings.createIdentity(fieldCount));
+    final RelNode newTableAccessRel = project(dq, fieldsUsed, extraFields, relBuilder);
+
+    // Some parts of the system can't handle rows with zero fields, so
+    // pretend that one field is used.
+    if (fieldsUsed.cardinality() == 0) {
+      RelNode input = newTableAccessRel;
+      if (input instanceof Project) {
+        // The table has implemented the project in the obvious way - by
+        // creating project with 0 fields. Strip it away, and create our own
+        // project with one field.
+        Project project = (Project) input;
+        if (project.getRowType().getFieldCount() == 0) {
+          input = project.getInput();
+        }
+      }
+      return dummyProject(fieldCount, input);
     }
 
-    relBuilder.push(newInput);
-    final int offset =
-        sort.offset == null ? 0 : RexLiteral.intValue(sort.offset);
-    final int fetch =
-        sort.fetch == null ? -1 : RexLiteral.intValue(sort.fetch);
-    final ImmutableList<RexNode> fields =
-        relBuilder.fields(RexUtil.apply(inputMapping, collation));
-
-    // The result has the same mapping as the input gave us. Sometimes we
-    // return fields that the consumer didn't ask for, because the filter
-    // needs them for its condition.
-    // TODO: Calcite will return empty LogicalValues when offset == 0 && fetch == 0.
-    // However, Hive ASTConverter can not deal with LogicalValues.
-    sortLimit(cluster, relBuilder, offset, fetch, fields);
-    return result(relBuilder.build(), inputMapping);
+    final Mapping mapping = createMapping(fieldsUsed, fieldCount);
+    return result(newTableAccessRel, mapping);
   }
-  
-  private List<RexNode> projects(RelDataType inputRowType, RelOptCluster cluster) {
-    final List<RexNode> exprList = new ArrayList<>();
-    for (RelDataTypeField field : inputRowType.getFieldList()) {
-      final RexBuilder rexBuilder = cluster.getRexBuilder();
-      exprList.add(rexBuilder.makeInputRef(field.getType(), field.getIndex()));
+
+  private static RelNode project(DruidQuery dq, ImmutableBitSet fieldsUsed,
+          Set<RelDataTypeField> extraFields, RelBuilder relBuilder) {
+    final int fieldCount = dq.getRowType().getFieldCount();
+    if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))
+        && extraFields.isEmpty()) {
+      return dq;
     }
-    return exprList;
-  }
-  
-  private static RelFieldCollation collation(RexNode node,
-      RelFieldCollation.Direction direction,
-      RelFieldCollation.NullDirection nullDirection, List<RexNode> extraNodes) {
-    switch (node.getKind()) {
-    case INPUT_REF:
-      return new RelFieldCollation(((RexInputRef) node).getIndex(), direction,
-          Util.first(nullDirection, direction.defaultNullDirection()));
-    case DESCENDING:
-      return collation(((RexCall) node).getOperands().get(0),
-          RelFieldCollation.Direction.DESCENDING,
-          nullDirection, extraNodes);
-    case NULLS_FIRST:
-      return collation(((RexCall) node).getOperands().get(0), direction,
-          RelFieldCollation.NullDirection.FIRST, extraNodes);
-    case NULLS_LAST:
-      return collation(((RexCall) node).getOperands().get(0), direction,
-          RelFieldCollation.NullDirection.LAST, extraNodes);
-    default:
-      final int fieldIndex = extraNodes.size();
-      extraNodes.add(node);
-      return new RelFieldCollation(fieldIndex, direction,
-          Util.first(nullDirection, direction.defaultNullDirection()));
+    final List<RexNode> exprList = new ArrayList<>();
+    final List<String> nameList = new ArrayList<>();
+    final RexBuilder rexBuilder = dq.getCluster().getRexBuilder();
+    final List<RelDataTypeField> fields = dq.getRowType().getFieldList();
+
+    // Project the subset of fields.
+    for (int i : fieldsUsed) {
+      RelDataTypeField field = fields.get(i);
+      exprList.add(rexBuilder.makeInputRef(dq, i));
+      nameList.add(field.getName());
     }
-  }
-  
- private void sortLimit(RelOptCluster cluster, RelBuilder relBuilder, int offset, int fetch,
-     Iterable<? extends RexNode> nodes) {
-   final List<RelFieldCollation> fieldCollations = new ArrayList<>();
-   final RelDataType inputRowType = relBuilder.peek().getRowType();
-   final List<RexNode> extraNodes = projects(inputRowType, cluster);
-   final List<RexNode> originalExtraNodes = ImmutableList.copyOf(extraNodes);
-   for (RexNode node : nodes) {
-     fieldCollations.add(
-         collation(node, RelFieldCollation.Direction.ASCENDING,
-                 RelFieldCollation.NullDirection.FIRST, extraNodes));
-   }
-   final RexNode offsetNode = offset <= 0 ? null : relBuilder.literal(offset);
-   final RexNode fetchNode = fetch < 0 ? null : relBuilder.literal(fetch);
-   if (offsetNode == null && fetchNode == null && fieldCollations.isEmpty()) {
-     return; // sort is trivial
-   }
 
-   final boolean addedFields = extraNodes.size() > originalExtraNodes.size();
-   if (fieldCollations.isEmpty()) {
-     assert !addedFields;
-     RelNode top = relBuilder.peek();
-     if (top instanceof Sort) {
-       final Sort sort2 = (Sort) top;
-       if (sort2.offset == null && sort2.fetch == null) {
-         relBuilder.build();
-         relBuilder.push(sort2.getInput());
-         final RelNode sort =
-             HiveSortLimit.create(relBuilder.build(), sort2.collation,
-                 offsetNode, fetchNode);
-         relBuilder.push(sort);
-         return;
-       }
-     }
-     if (top instanceof Project) {
-       final Project project = (Project) top;
-       if (project.getInput() instanceof Sort) {
-         final Sort sort2 = (Sort) project.getInput();
-         if (sort2.offset == null && sort2.fetch == null) {
-           relBuilder.build();
-           relBuilder.push(sort2.getInput());
-           final RelNode sort =
-               HiveSortLimit.create(relBuilder.build(), sort2.collation,
-                   offsetNode, fetchNode);
-           relBuilder.push(sort);
-           relBuilder.project(project.getProjects());
-           return;
-         }
-       }
-     }
-   }
-   if (addedFields) {
-     relBuilder.project(extraNodes);
-   }
-   final RelNode sort =
-       HiveSortLimit.create(relBuilder.build(), RelCollations.of(fieldCollations),
-           offsetNode, fetchNode);
-   relBuilder.push(sort);
-   if (addedFields) {
-     relBuilder.project(originalExtraNodes);
-   }
-   return;
- }
- 
-  private TrimResult result(RelNode r, final Mapping mapping) {
-    final RexBuilder rexBuilder = relBuilder.getRexBuilder();
-    for (final CorrelationId correlation : r.getVariablesSet()) {
-      r = r.accept(
-          new CorrelationReferenceFinder() {
-            @Override
-            protected RexNode handle(RexFieldAccess fieldAccess) {
-              final RexCorrelVariable v =
-                  (RexCorrelVariable) fieldAccess.getReferenceExpr();
-              if (v.id.equals(correlation)
-                  && v.getType().getFieldCount() == mapping.getSourceCount()) {
-                final int old = fieldAccess.getField().getIndex();
-                final int new_ = mapping.getTarget(old);
-                final RelDataTypeFactory.FieldInfoBuilder typeBuilder =
-                    relBuilder.getTypeFactory().builder();
-                for (int target : Util.range(mapping.getTargetCount())) {
-                  typeBuilder.add(
-                      v.getType().getFieldList().get(mapping.getSource(target)));
-                }
-                final RexNode newV =
-                    rexBuilder.makeCorrel(typeBuilder.build(), v.id);
-                if (old != new_) {
-                  return rexBuilder.makeFieldAccess(newV, new_);
-                }
-              }
-              return fieldAccess;
-            }
-
-          });
+    // Project nulls for the extra fields. (Maybe a sub-class table has
+    // extra fields, but we don't.)
+    for (RelDataTypeField extraField : extraFields) {
+      exprList.add(
+          rexBuilder.ensureType(
+              extraField.getType(),
+              rexBuilder.constantNull(),
+              true));
+      nameList.add(extraField.getName());
     }
-    return new TrimResult(r, mapping);
+
+    HiveProject hp = (HiveProject) relBuilder.push(dq).project(exprList, nameList).build();
+    hp.setSynthetic();
+    return hp;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdCollation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdCollation.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdCollation.java
index 18fe650..b86b947 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdCollation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdCollation.java
@@ -20,7 +20,10 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.stats;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
 import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -32,7 +35,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 
 import com.google.common.collect.ImmutableList;
 
-public class HiveRelMdCollation {
+public class HiveRelMdCollation implements MetadataHandler<BuiltInMetadata.Collation> {
 
   public static final RelMetadataProvider SOURCE =
           ChainedRelMetadataProvider.of(
@@ -47,6 +50,11 @@ public class HiveRelMdCollation {
 
   //~ Methods ----------------------------------------------------------------
 
+  @Override
+  public MetadataDef<BuiltInMetadata.Collation> getDef() {
+    return BuiltInMetadata.Collation.DEF;
+  }
+
   public ImmutableList<RelCollation> collations(HiveAggregate aggregate, RelMetadataQuery mq) {
     // Compute collations
     ImmutableList.Builder<RelFieldCollation> collationListBuilder =

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistribution.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistribution.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistribution.java
index 62d3ead..e574e7a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistribution.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistribution.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.hive.ql.optimizer.calcite.stats;
 
 import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
 import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
-import org.apache.calcite.rel.metadata.RelMdDistribution;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.BuiltInMethod;
@@ -30,7 +32,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 
 import com.google.common.collect.ImmutableList;
 
-public class HiveRelMdDistribution {
+public class HiveRelMdDistribution implements MetadataHandler<BuiltInMetadata.Distribution> {
 
   public static final RelMetadataProvider SOURCE =
           ChainedRelMetadataProvider.of(
@@ -44,6 +46,10 @@ public class HiveRelMdDistribution {
 
   //~ Methods ----------------------------------------------------------------
 
+  public MetadataDef<BuiltInMetadata.Distribution> getDef() {
+    return BuiltInMetadata.Distribution.DEF;
+  }
+
   public RelDistribution distribution(HiveAggregate aggregate, RelMetadataQuery mq) {
     return new HiveRelDistribution(RelDistribution.Type.HASH_DISTRIBUTED,
             aggregate.getGroupSet().asList());

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
index e468573..69e157e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
@@ -40,6 +40,10 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.SemiJoin;
 import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
 import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMdPredicates;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -72,14 +76,28 @@ import com.google.common.collect.Maps;
 
 
 //TODO: Move this to calcite
-public class HiveRelMdPredicates extends RelMdPredicates {
+public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Predicates> {
 
-  public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(
-                                                     BuiltInMethod.PREDICATES.method,
-                                                     new HiveRelMdPredicates());
+  public static final RelMetadataProvider SOURCE =
+          ChainedRelMetadataProvider.of(
+                  ImmutableList.of(
+                          ReflectiveRelMetadataProvider.reflectiveSource(
+                                  BuiltInMethod.PREDICATES.method, new HiveRelMdPredicates()),
+                          RelMdPredicates.SOURCE));
 
   private static final List<RexNode> EMPTY_LIST = ImmutableList.of();
 
+  //~ Constructors -----------------------------------------------------------
+
+  private HiveRelMdPredicates() {}
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override
+  public MetadataDef<BuiltInMetadata.Predicates> getDef() {
+    return BuiltInMetadata.Predicates.DEF;
+  }
+
   /**
    * Infers predicates for a project.
    *
@@ -99,8 +117,8 @@ public class HiveRelMdPredicates extends RelMdPredicates {
    *
    * </ol>
    */
-  @Override
   public RelOptPredicateList getPredicates(Project project, RelMetadataQuery mq) {
+
     RelNode child = project.getInput();
     final RexBuilder rexBuilder = project.getCluster().getRexBuilder();
     RelOptPredicateList childInfo = mq.getPulledUpPredicates(child);
@@ -151,7 +169,6 @@ public class HiveRelMdPredicates extends RelMdPredicates {
   }
 
   /** Infers predicates for a {@link org.apache.calcite.rel.core.Join}. */
-  @Override
   public RelOptPredicateList getPredicates(Join join, RelMetadataQuery mq) {
     RexBuilder rB = join.getCluster().getRexBuilder();
     RelNode left = join.getInput(0);
@@ -181,7 +198,6 @@ public class HiveRelMdPredicates extends RelMdPredicates {
    * pulledUpExprs    : { a &gt; 7}
    * </pre>
    */
-  @Override
   public RelOptPredicateList getPredicates(Aggregate agg, RelMetadataQuery mq) {
     final RelNode input = agg.getInput();
     final RelOptPredicateList inputInfo = mq.getPulledUpPredicates(input);
@@ -209,7 +225,6 @@ public class HiveRelMdPredicates extends RelMdPredicates {
   /**
    * Infers predicates for a Union.
    */
-  @Override
   public RelOptPredicateList getPredicates(Union union, RelMetadataQuery mq) {
     RexBuilder rB = union.getCluster().getRexBuilder();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
index 0d03ebb..651adc0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
@@ -41,13 +41,16 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import com.google.common.collect.ImmutableMap;
 
 public class HiveRelMdSelectivity extends RelMdSelectivity {
-  public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(
-                                                     BuiltInMethod.SELECTIVITY.method,
-                                                     new HiveRelMdSelectivity());
 
-  protected HiveRelMdSelectivity() {
-    super();
-  }
+  public static final RelMetadataProvider SOURCE =
+      ReflectiveRelMetadataProvider.reflectiveSource(
+          BuiltInMethod.SELECTIVITY.method, new HiveRelMdSelectivity());
+
+  //~ Constructors -----------------------------------------------------------
+
+  private HiveRelMdSelectivity() {}
+
+  //~ Methods ----------------------------------------------------------------
 
   public Double getSelectivity(HiveTableScan t, RelMetadataQuery mq, RexNode predicate) {
     if (predicate != null) {
@@ -58,7 +61,7 @@ public class HiveRelMdSelectivity extends RelMdSelectivity {
     return 1.0;
   }
 
-  public Double getSelectivity(HiveJoin j, RelMetadataQuery mq, RexNode predicate) throws CalciteSemanticException {
+  public Double getSelectivity(HiveJoin j, RelMetadataQuery mq, RexNode predicate) {
     if (j.getJoinType().equals(JoinRelType.INNER)) {
       return computeInnerJoinSelectivity(j, mq, predicate);
     } else if (j.getJoinType().equals(JoinRelType.LEFT) ||
@@ -75,7 +78,7 @@ public class HiveRelMdSelectivity extends RelMdSelectivity {
     return 1.0;
   }
 
-  private Double computeInnerJoinSelectivity(HiveJoin j, RelMetadataQuery mq, RexNode predicate) throws CalciteSemanticException {
+  private Double computeInnerJoinSelectivity(HiveJoin j, RelMetadataQuery mq, RexNode predicate) {
     double ndvCrossProduct = 1;
     Pair<Boolean, RexNode> predInfo =
         getCombinedPredicateForJoin(j, predicate);
@@ -86,8 +89,13 @@ public class HiveRelMdSelectivity extends RelMdSelectivity {
     }
 
     RexNode combinedPredicate = predInfo.getValue();
-    JoinPredicateInfo jpi = JoinPredicateInfo.constructJoinPredicateInfo(j,
-        combinedPredicate);
+    JoinPredicateInfo jpi;
+    try {
+      jpi = JoinPredicateInfo.constructJoinPredicateInfo(j,
+          combinedPredicate);
+    } catch (CalciteSemanticException e) {
+      throw new RuntimeException(e);
+    }
     ImmutableMap.Builder<Integer, Double> colStatMapBuilder = ImmutableMap
         .builder();
     ImmutableMap<Integer, Double> colStatMap;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
index 31adb41..1039f56 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
@@ -122,7 +122,18 @@ public class HiveRelMdSize extends RelMdSize {
     case BIGINT:
     case DOUBLE:
     case TIMESTAMP:
-    case INTERVAL_DAY_TIME:
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_MONTH:
+    case INTERVAL_SECOND:
+    case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
       return 8d;
     case BINARY:

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdUniqueKeys.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdUniqueKeys.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdUniqueKeys.java
index 0718150..9a5a2ba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdUniqueKeys.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdUniqueKeys.java
@@ -30,9 +30,9 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.metadata.BuiltInMetadata;
-import org.apache.calcite.rel.metadata.Metadata;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
 import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
-import org.apache.calcite.rel.metadata.RelMdUniqueKeys;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexInputRef;
@@ -43,13 +43,16 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.plan.ColStatistics;
 
-import com.google.common.base.Function;
+public class HiveRelMdUniqueKeys implements MetadataHandler<BuiltInMetadata.UniqueKeys> {
 
-public class HiveRelMdUniqueKeys {
+  public static final RelMetadataProvider SOURCE =
+      ReflectiveRelMetadataProvider.reflectiveSource(
+          BuiltInMethod.UNIQUE_KEYS.method, new HiveRelMdUniqueKeys());
 
-  public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider
-      .reflectiveSource(BuiltInMethod.UNIQUE_KEYS.method,
-          new HiveRelMdUniqueKeys());
+  @Override
+  public MetadataDef<BuiltInMetadata.UniqueKeys> getDef() {
+    return BuiltInMetadata.UniqueKeys.DEF;
+  }
 
   /*
    * Infer Uniquenes if: - rowCount(col) = ndv(col) - TBD for numerics: max(col)
@@ -65,7 +68,60 @@ public class HiveRelMdUniqueKeys {
     HiveTableScan tScan = getTableScan(rel.getInput(), false);
 
     if (tScan == null) {
-      return mq.getUniqueKeys(rel, ignoreNulls);
+      // If HiveTableScan is not found, e.g., not sequence of Project and
+      // Filter operators, execute the original getUniqueKeys method
+
+      // LogicalProject maps a set of rows to a different set;
+      // Without knowledge of the mapping function(whether it
+      // preserves uniqueness), it is only safe to derive uniqueness
+      // info from the child of a project when the mapping is f(a) => a.
+      //
+      // Further more, the unique bitset coming from the child needs
+      // to be mapped to match the output of the project.
+      final Map<Integer, Integer> mapInToOutPos = new HashMap<>();
+      final List<RexNode> projExprs = rel.getProjects();
+      final Set<ImmutableBitSet> projUniqueKeySet = new HashSet<>();
+
+      // Build an input to output position map.
+      for (int i = 0; i < projExprs.size(); i++) {
+        RexNode projExpr = projExprs.get(i);
+        if (projExpr instanceof RexInputRef) {
+          mapInToOutPos.put(((RexInputRef) projExpr).getIndex(), i);
+        }
+      }
+
+      if (mapInToOutPos.isEmpty()) {
+        // if there's no RexInputRef in the projected expressions
+        // return empty set.
+        return projUniqueKeySet;
+      }
+
+      Set<ImmutableBitSet> childUniqueKeySet =
+          mq.getUniqueKeys(rel.getInput(), ignoreNulls);
+
+      if (childUniqueKeySet != null) {
+        // Now add to the projUniqueKeySet the child keys that are fully
+        // projected.
+        for (ImmutableBitSet colMask : childUniqueKeySet) {
+          ImmutableBitSet.Builder tmpMask = ImmutableBitSet.builder();
+          boolean completeKeyProjected = true;
+          for (int bit : colMask) {
+            if (mapInToOutPos.containsKey(bit)) {
+              tmpMask.set(mapInToOutPos.get(bit));
+            } else {
+              // Skip the child unique key if part of it is not
+              // projected.
+              completeKeyProjected = false;
+              break;
+            }
+          }
+          if (completeKeyProjected) {
+            projUniqueKeySet.add(tmpMask.build());
+          }
+        }
+      }
+
+      return projUniqueKeySet;
     }
 
     Map<Integer, Integer> posMap = new HashMap<Integer, Integer>();

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
index 9a5becb..7b9ee84 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
@@ -22,15 +22,15 @@ import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.Calendar;
 
+import org.apache.calcite.adapter.druid.DruidQuery;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
@@ -58,19 +58,20 @@ class ASTBuilder {
                 "TOK_TMP_FILE")).node();
   }
 
-  static ASTNode table(TableScan scan) {
-    RelOptHiveTable hTbl = (RelOptHiveTable) scan.getTable();
-    ASTBuilder b = ASTBuilder.construct(HiveParser.TOK_TABREF, "TOK_TABREF").add(
-        ASTBuilder.construct(HiveParser.TOK_TABNAME, "TOK_TABNAME")
-            .add(HiveParser.Identifier, hTbl.getHiveTableMD().getDbName())
-            .add(HiveParser.Identifier, hTbl.getHiveTableMD().getTableName()));
-
+  static ASTNode table(RelNode scan) {
     HiveTableScan hts;
     if (scan instanceof DruidQuery) {
       hts = (HiveTableScan) ((DruidQuery)scan).getTableScan();
     } else {
       hts = (HiveTableScan) scan;
     }
+
+    RelOptHiveTable hTbl = (RelOptHiveTable) hts.getTable();
+    ASTBuilder b = ASTBuilder.construct(HiveParser.TOK_TABREF, "TOK_TABREF").add(
+        ASTBuilder.construct(HiveParser.TOK_TABNAME, "TOK_TABNAME")
+            .add(HiveParser.Identifier, hTbl.getHiveTableMD().getDbName())
+            .add(HiveParser.Identifier, hTbl.getHiveTableMD().getTableName()));
+
     ASTBuilder propList = ASTBuilder.construct(HiveParser.TOK_TABLEPROPLIST, "TOK_TABLEPROPLIST");
     if (scan instanceof DruidQuery) {
       // Pass possible query to Druid
@@ -181,8 +182,19 @@ class ASTBuilder {
     case DATE:
     case TIME:
     case TIMESTAMP:
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_MONTH:
+    case INTERVAL_SECOND:
+    case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
-    case INTERVAL_DAY_TIME:
       if (literal.getValue() == null) {
         return ASTBuilder.construct(HiveParser.TOK_NULL, "TOK_NULL").node();
       }
@@ -273,14 +285,25 @@ class ASTBuilder {
       val = "'" + val + "'";
     }
       break;
+    case INTERVAL_YEAR:
+    case INTERVAL_MONTH:
     case INTERVAL_YEAR_MONTH: {
       type = HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL;
       BigDecimal monthsBd = (BigDecimal) literal.getValue();
       HiveIntervalYearMonth intervalYearMonth = new HiveIntervalYearMonth(monthsBd.intValue());
       val = "'" + intervalYearMonth.toString() + "'";
-      break;
     }
-    case INTERVAL_DAY_TIME: {
+      break;
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_SECOND: {
       type = HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL;
       BigDecimal millisBd = (BigDecimal) literal.getValue();
 
@@ -288,8 +311,8 @@ class ASTBuilder {
       BigDecimal secsBd = millisBd.divide(BigDecimal.valueOf(1000));
       HiveIntervalDayTime intervalDayTime = new HiveIntervalDayTime(secsBd);
       val = "'" + intervalDayTime.toString() + "'";
-      break;
     }
+      break;
     case NULL:
       type = HiveParser.TOK_NULL;
       break;


[3/6] hive git commit: HIVE-13316: Upgrade to Calcite 1.10 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index 8d738aa..63aa086 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -23,6 +23,8 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.calcite.adapter.druid.DruidQuery;
+import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelVisitor;
@@ -57,8 +59,9 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExtractDate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFloorDate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableFunctionScan;
@@ -346,6 +349,10 @@ public class ASTConverter {
       TableScan f = (TableScan) r;
       s = new Schema(f);
       ast = ASTBuilder.table(f);
+    } else if (r instanceof DruidQuery) {
+      DruidQuery f = (DruidQuery) r;
+      s = new Schema(f);
+      ast = ASTBuilder.table(f);
     } else if (r instanceof Join) {
       Join join = (Join) r;
       QueryBlockInfo left = convertSource(join.getLeft());
@@ -425,7 +432,8 @@ public class ASTConverter {
     @Override
     public void visit(RelNode node, int ordinal, RelNode parent) {
 
-      if (node instanceof TableScan) {
+      if (node instanceof TableScan ||
+          node instanceof DruidQuery) {
         ASTConverter.this.from = node;
       } else if (node instanceof Filter) {
         handle((Filter) node);
@@ -645,14 +653,30 @@ public class ASTConverter {
         astNodeLst.add(astBldr.node());
       }
 
-      for (RexNode operand : call.operands) {
-        astNodeLst.add(operand.accept(this));
+      if (op.kind == SqlKind.EXTRACT) {
+        // Extract on date: special handling since function in Hive does
+        // include <time_unit>. Observe that <time_unit> information
+        // is implicit in the function name, thus translation will
+        // proceed correctly if we just ignore the <time_unit>
+        astNodeLst.add(call.operands.get(1).accept(this));
+      } else if (op.kind == SqlKind.FLOOR &&
+              call.operands.size() == 2) {
+        // Floor on date: special handling since function in Hive does
+        // include <time_unit>. Observe that <time_unit> information
+        // is implicit in the function name, thus translation will
+        // proceed correctly if we just ignore the <time_unit>
+        astNodeLst.add(call.operands.get(0).accept(this));
+      } else {
+        for (RexNode operand : call.operands) {
+          astNodeLst.add(operand.accept(this));
+        }
       }
 
-      if (isFlat(call))
+      if (isFlat(call)) {
         return SqlFunctionConverter.buildAST(op, astNodeLst, 0);
-      else
+      } else {
         return SqlFunctionConverter.buildAST(op, astNodeLst);
+      }
     }
   }
 
@@ -675,18 +699,21 @@ public class ASTConverter {
     private static final long serialVersionUID = 1L;
 
     Schema(TableScan scan) {
-      HiveTableScan hts;
-      if (scan instanceof DruidQuery) {
-        hts = (HiveTableScan) ((DruidQuery)scan).getTableScan();
-      } else {
-        hts = (HiveTableScan) scan;
-      }
+      HiveTableScan hts = (HiveTableScan) scan;
       String tabName = hts.getTableAlias();
       for (RelDataTypeField field : scan.getRowType().getFieldList()) {
         add(new ColumnInfo(tabName, field.getName()));
       }
     }
 
+    Schema(DruidQuery dq) {
+      HiveTableScan hts = (HiveTableScan) ((DruidQuery)dq).getTableScan();
+      String tabName = hts.getTableAlias();
+      for (RelDataTypeField field : dq.getRowType().getFieldList()) {
+        add(new ColumnInfo(tabName, field.getName()));
+      }
+    }
+
     Schema(Project select, String alias) {
       for (RelDataTypeField field : select.getRowType().getFieldList()) {
         add(new ColumnInfo(alias, field.getName()));

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
index 46b936a..2d621e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
@@ -46,13 +46,11 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
-import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcFactory;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.RexVisitor;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.Schema;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
-import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderExpression;
@@ -75,7 +73,6 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -168,9 +165,23 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
     }
 
     List<ExprNodeDesc> args = new LinkedList<ExprNodeDesc>();
-
-    for (RexNode operand : call.operands) {
-      args.add(operand.accept(this));
+    if (call.getKind() == SqlKind.EXTRACT) {
+      // Extract on date: special handling since function in Hive does
+      // include <time_unit>. Observe that <time_unit> information
+      // is implicit in the function name, thus translation will
+      // proceed correctly if we just ignore the <time_unit>
+      args.add(call.operands.get(1).accept(this));
+    } else if (call.getKind() == SqlKind.FLOOR &&
+            call.operands.size() == 2) {
+      // Floor on date: special handling since function in Hive does
+      // include <time_unit>. Observe that <time_unit> information
+      // is implicit in the function name, thus translation will
+      // proceed correctly if we just ignore the <time_unit>
+      args.add(call.operands.get(0).accept(this));
+    } else {
+      for (RexNode operand : call.operands) {
+        args.add(operand.accept(this));
+      }
     }
 
     // If Call is a redundant cast then bail out. Ex: cast(true)BOOLEAN
@@ -239,9 +250,20 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
       case VARCHAR:
       case CHAR:
         return new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, null);
+      case INTERVAL_YEAR:
+      case INTERVAL_MONTH:
       case INTERVAL_YEAR_MONTH:
         return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo, null);
-      case INTERVAL_DAY_TIME:
+      case INTERVAL_DAY:
+      case INTERVAL_DAY_HOUR:
+      case INTERVAL_DAY_MINUTE:
+      case INTERVAL_DAY_SECOND:
+      case INTERVAL_HOUR:
+      case INTERVAL_HOUR_MINUTE:
+      case INTERVAL_HOUR_SECOND:
+      case INTERVAL_MINUTE:
+      case INTERVAL_MINUTE_SECOND:
+      case INTERVAL_SECOND:
         return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo, null);
       case OTHER:
       default:
@@ -291,12 +313,23 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
       case CHAR: {
         return new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, literal.getValue3());
       }
+      case INTERVAL_YEAR:
+      case INTERVAL_MONTH:
       case INTERVAL_YEAR_MONTH: {
         BigDecimal monthsBd = (BigDecimal) literal.getValue();
         return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,
                 new HiveIntervalYearMonth(monthsBd.intValue()));
       }
-      case INTERVAL_DAY_TIME: {
+      case INTERVAL_DAY:
+      case INTERVAL_DAY_HOUR:
+      case INTERVAL_DAY_MINUTE:
+      case INTERVAL_DAY_SECOND:
+      case INTERVAL_HOUR:
+      case INTERVAL_HOUR_MINUTE:
+      case INTERVAL_HOUR_SECOND:
+      case INTERVAL_MINUTE:
+      case INTERVAL_MINUTE_SECOND:
+      case INTERVAL_SECOND: {
         BigDecimal millisBd = (BigDecimal) literal.getValue();
         // Calcite literal is in millis, we need to convert to seconds
         BigDecimal secsBd = millisBd.divide(BigDecimal.valueOf(1000));

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index 9db7727..acc2799 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.translator;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.calcite.adapter.druid.DruidQuery;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.hep.HepRelVertex;
 import org.apache.calcite.plan.volcano.RelSubset;
@@ -111,6 +112,10 @@ public class PlanModifierForASTConv {
     if (rel instanceof HiveTableScan) {
       return ((HiveTableScan)rel).getTableAlias();
     }
+    if (rel instanceof DruidQuery) {
+      DruidQuery dq = (DruidQuery) rel;
+      return ((HiveTableScan) dq.getTableScan()).getTableAlias();
+    }
     if (rel instanceof Project) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
index 479070b..cb7bc26 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
@@ -55,6 +56,8 @@ import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException.UnsupportedFeature;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExtractDate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFloorDate;
 import org.apache.hadoop.hive.ql.parse.ParseUtils;
 import org.apache.hadoop.hive.ql.parse.RowResolver;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -230,9 +233,15 @@ public class RexNodeConverter {
       retType = TypeConverter.convert(func.getTypeInfo(), cluster.getTypeFactory());
       SqlOperator calciteOp = SqlFunctionConverter.getCalciteOperator(func.getFuncText(),
           func.getGenericUDF(), argTypeBldr.build(), retType);
-      // If it is a case operator, we need to rewrite it
       if (calciteOp.getKind() == SqlKind.CASE) {
+        // If it is a case operator, we need to rewrite it
         childRexNodeLst = rewriteCaseChildren(func, childRexNodeLst);
+      } else if (HiveExtractDate.ALL_FUNCTIONS.contains(calciteOp)) {
+        // If it is a extract operator, we need to rewrite it
+        childRexNodeLst = rewriteExtractDateChildren(calciteOp, childRexNodeLst);
+      } else if (HiveFloorDate.ALL_FUNCTIONS.contains(calciteOp)) {
+        // If it is a floor <date> operator, we need to rewrite it
+        childRexNodeLst = rewriteFloorDateChildren(calciteOp, childRexNodeLst);
       }
       expr = cluster.getRexBuilder().makeCall(calciteOp, childRexNodeLst);
     } else {
@@ -340,6 +349,56 @@ public class RexNodeConverter {
     return newChildRexNodeLst;
   }
 
+  private List<RexNode> rewriteExtractDateChildren(SqlOperator op, List<RexNode> childRexNodeLst)
+      throws SemanticException {
+    List<RexNode> newChildRexNodeLst = new ArrayList<RexNode>();
+    if (op == HiveExtractDate.YEAR) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.YEAR));
+    } else if (op == HiveExtractDate.QUARTER) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.QUARTER));
+    } else if (op == HiveExtractDate.MONTH) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.MONTH));
+    } else if (op == HiveExtractDate.WEEK) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.WEEK));
+    } else if (op == HiveExtractDate.DAY) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.DAY));
+    } else if (op == HiveExtractDate.HOUR) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.HOUR));
+    } else if (op == HiveExtractDate.MINUTE) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.MINUTE));
+    } else if (op == HiveExtractDate.SECOND) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.SECOND));
+    }
+    assert childRexNodeLst.size() == 1;
+    newChildRexNodeLst.add(childRexNodeLst.get(0));
+    return newChildRexNodeLst;
+  }
+
+  private List<RexNode> rewriteFloorDateChildren(SqlOperator op, List<RexNode> childRexNodeLst)
+      throws SemanticException {
+    List<RexNode> newChildRexNodeLst = new ArrayList<RexNode>();
+    assert childRexNodeLst.size() == 1;
+    newChildRexNodeLst.add(childRexNodeLst.get(0));
+    if (op == HiveFloorDate.YEAR) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.YEAR));
+    } else if (op == HiveFloorDate.QUARTER) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.QUARTER));
+    } else if (op == HiveFloorDate.MONTH) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.MONTH));
+    } else if (op == HiveFloorDate.WEEK) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.WEEK));
+    } else if (op == HiveFloorDate.DAY) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.DAY));
+    } else if (op == HiveFloorDate.HOUR) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.HOUR));
+    } else if (op == HiveFloorDate.MINUTE) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.MINUTE));
+    } else if (op == HiveFloorDate.SECOND) {
+      newChildRexNodeLst.add(cluster.getRexBuilder().makeFlag(TimeUnitRange.SECOND));
+    }
+    return newChildRexNodeLst;
+  }
+
   private static boolean checkForStatefulFunctions(List<ExprNodeDesc> list) {
     for (ExprNodeDesc node : list) {
       if (node instanceof ExprNodeGenericFuncDesc) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index f150132..cb86934 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -48,7 +48,8 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlCountAggFunc
 import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlMinMaxAggFunction;
 import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlSumAggFunction;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveBetween;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExtractDate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFloorDate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIn;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
@@ -216,6 +217,8 @@ public class SqlFunctionConverter {
         case IS_NOT_NULL:
         case IS_NULL:
         case CASE:
+        case EXTRACT:
+        case FLOOR:
         case OTHER_FUNCTION:
           node = (ASTNode) ParseDriver.adaptor.create(HiveParser.TOK_FUNCTION, "TOK_FUNCTION");
           node.addChild((ASTNode) ParseDriver.adaptor.create(hToken.type, hToken.text));
@@ -346,21 +349,37 @@ public class SqlFunctionConverter {
       registerFunction("when", SqlStdOperatorTable.CASE, hToken(HiveParser.Identifier, "when"));
       registerDuplicateFunction("case", SqlStdOperatorTable.CASE, hToken(HiveParser.Identifier, "when"));
       // timebased
-      registerFunction("floor_year", HiveDateGranularity.YEAR,
+      registerFunction("year", HiveExtractDate.YEAR,
+          hToken(HiveParser.Identifier, "year"));
+      registerFunction("quarter", HiveExtractDate.QUARTER,
+          hToken(HiveParser.Identifier, "quarter"));
+      registerFunction("month", HiveExtractDate.MONTH,
+          hToken(HiveParser.Identifier, "month"));
+      registerFunction("weekofyear", HiveExtractDate.WEEK,
+          hToken(HiveParser.Identifier, "weekofyear"));
+      registerFunction("day", HiveExtractDate.DAY,
+          hToken(HiveParser.Identifier, "day"));
+      registerFunction("hour", HiveExtractDate.HOUR,
+          hToken(HiveParser.Identifier, "hour"));
+      registerFunction("minute", HiveExtractDate.MINUTE,
+          hToken(HiveParser.Identifier, "minute"));
+      registerFunction("second", HiveExtractDate.SECOND,
+          hToken(HiveParser.Identifier, "second"));
+      registerFunction("floor_year", HiveFloorDate.YEAR,
           hToken(HiveParser.Identifier, "floor_year"));
-      registerFunction("floor_quarter", HiveDateGranularity.QUARTER,
+      registerFunction("floor_quarter", HiveFloorDate.QUARTER,
           hToken(HiveParser.Identifier, "floor_quarter"));
-      registerFunction("floor_month", HiveDateGranularity.MONTH,
+      registerFunction("floor_month", HiveFloorDate.MONTH,
           hToken(HiveParser.Identifier, "floor_month"));
-      registerFunction("floor_week", HiveDateGranularity.WEEK,
+      registerFunction("floor_week", HiveFloorDate.WEEK,
           hToken(HiveParser.Identifier, "floor_week"));
-      registerFunction("floor_day", HiveDateGranularity.DAY,
+      registerFunction("floor_day", HiveFloorDate.DAY,
           hToken(HiveParser.Identifier, "floor_day"));
-      registerFunction("floor_hour", HiveDateGranularity.HOUR,
+      registerFunction("floor_hour", HiveFloorDate.HOUR,
           hToken(HiveParser.Identifier, "floor_hour"));
-      registerFunction("floor_minute", HiveDateGranularity.MINUTE,
+      registerFunction("floor_minute", HiveFloorDate.MINUTE,
           hToken(HiveParser.Identifier, "floor_minute"));
-      registerFunction("floor_second", HiveDateGranularity.SECOND,
+      registerFunction("floor_second", HiveFloorDate.SECOND,
           hToken(HiveParser.Identifier, "floor_second"));
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java
index ba41518..a47010d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java
@@ -75,8 +75,32 @@ public class TypeConverter {
     b.put(SqlTypeName.DOUBLE.getName(), new HiveToken(HiveParser.TOK_DOUBLE, "TOK_DOUBLE"));
     b.put(SqlTypeName.DATE.getName(), new HiveToken(HiveParser.TOK_DATE, "TOK_DATE"));
     b.put(SqlTypeName.TIMESTAMP.getName(), new HiveToken(HiveParser.TOK_TIMESTAMP, "TOK_TIMESTAMP"));
-    b.put(SqlTypeName.INTERVAL_YEAR_MONTH.getName(), new HiveToken(HiveParser.TOK_INTERVAL_YEAR_MONTH, "TOK_INTERVAL_YEAR_MONTH"));
-    b.put(SqlTypeName.INTERVAL_DAY_TIME.getName(), new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME, "TOK_INTERVAL_DAY_TIME"));
+    b.put(SqlTypeName.INTERVAL_YEAR.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL, "TOK_INTERVAL_YEAR_MONTH_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_MONTH.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL, "TOK_INTERVAL_YEAR_MONTH_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_YEAR_MONTH.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_YEAR_MONTH_LITERAL, "TOK_INTERVAL_YEAR_MONTH_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_DAY.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_DAY_HOUR.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_DAY_MINUTE.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_DAY_SECOND.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_HOUR.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_HOUR_MINUTE.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_HOUR_SECOND.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_MINUTE.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_MINUTE_SECOND.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
+    b.put(SqlTypeName.INTERVAL_SECOND.getName(),
+            new HiveToken(HiveParser.TOK_INTERVAL_DAY_TIME_LITERAL, "TOK_INTERVAL_DAY_TIME_LITERAL"));
     b.put(SqlTypeName.BINARY.getName(), new HiveToken(HiveParser.TOK_BINARY, "TOK_BINARY"));
     calciteToHiveTypeNameMap = b.build();
   };
@@ -301,9 +325,20 @@ public class TypeConverter {
       return TypeInfoFactory.dateTypeInfo;
     case TIMESTAMP:
       return TypeInfoFactory.timestampTypeInfo;
+    case INTERVAL_YEAR:
+    case INTERVAL_MONTH:
     case INTERVAL_YEAR_MONTH:
       return TypeInfoFactory.intervalYearMonthTypeInfo;
-    case INTERVAL_DAY_TIME:
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_SECOND:
       return TypeInfoFactory.intervalDayTimeTypeInfo;
     case BINARY:
       return TypeInfoFactory.binaryTypeInfo;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index cf66ad9..d32a0a7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -41,6 +41,10 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.antlr.runtime.ClassicToken;
 import org.antlr.runtime.tree.TreeVisitor;
 import org.antlr.runtime.tree.TreeVisitorAction;
+import org.apache.calcite.adapter.druid.DruidQuery;
+import org.apache.calcite.adapter.druid.DruidRules;
+import org.apache.calcite.adapter.druid.DruidSchema;
+import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptPlanner.Executor;
@@ -65,7 +69,9 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.calcite.rel.rules.JoinToMultiJoinRule;
 import org.apache.calcite.rel.rules.LoptOptimizeJoinRule;
@@ -78,6 +84,7 @@ import org.apache.calcite.rel.rules.UnionMergeRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexFieldCollation;
 import org.apache.calcite.rex.RexInputRef;
@@ -132,11 +139,6 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveAlgorithmsConf;
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveVolcanoPlanner;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidRules;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidSchema;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.HiveDruidConf;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
@@ -974,11 +976,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
       final Double maxMemory = (double) HiveConf.getLongVar(
               conf, HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD);
       HiveAlgorithmsConf algorithmsConf = new HiveAlgorithmsConf(maxSplitSize, maxMemory);
-      final int selectThreshold = (int) HiveConf.getIntVar(
-              conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_THRESHOLD);
-      HiveDruidConf druidConf = new HiveDruidConf(selectThreshold);
       HiveRulesRegistry registry = new HiveRulesRegistry();
-      HivePlannerContext confContext = new HivePlannerContext(algorithmsConf, druidConf, registry);
+      HivePlannerContext confContext = new HivePlannerContext(algorithmsConf, registry);
       RelOptPlanner planner = HiveVolcanoPlanner.createPlanner(confContext);
       final RelOptQuery query = new RelOptQuery(planner);
       final RexBuilder rexBuilder = cluster.getRexBuilder();
@@ -1008,8 +1007,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
           this.viewProjectToTableSchema);
       fieldTrimmer.trim(calciteGenPlan);
 
-      // Create MD provider
+      // Create and set MD provider
       HiveDefaultRelMetadataProvider mdProvider = new HiveDefaultRelMetadataProvider(conf);
+      RelMetadataQuery.THREAD_PROVIDERS.set(
+              JaninoRelMetadataProvider.of(mdProvider.getMetadataProvider()));
 
       // Create executor
       Executor executorProvider = new HiveRexExecutorImpl(cluster);
@@ -1031,8 +1032,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
           HepProgramBuilder hepPgmBldr = new HepProgramBuilder().addMatchOrder(HepMatchOrder.BOTTOM_UP);
           hepPgmBldr.addRuleInstance(new JoinToMultiJoinRule(HiveJoin.class));
-          hepPgmBldr.addRuleInstance(new LoptOptimizeJoinRule(HiveRelFactories.HIVE_JOIN_FACTORY,
-              HiveRelFactories.HIVE_PROJECT_FACTORY, HiveRelFactories.HIVE_FILTER_FACTORY));
+          hepPgmBldr.addRuleInstance(new LoptOptimizeJoinRule(HiveRelFactories.HIVE_BUILDER));
 
           HepProgram hepPgm = hepPgmBldr.build();
           HepPlanner hepPlanner = new HepPlanner(hepPgm);
@@ -1140,10 +1140,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
         calciteOptimizedPlan = fieldTrimmer.trim(calciteOptimizedPlan);
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
                 HepMatchOrder.BOTTOM_UP, ProjectRemoveRule.INSTANCE,
-                new ProjectMergeRule(false, HiveRelFactories.HIVE_PROJECT_FACTORY));
+                new ProjectMergeRule(false, HiveRelFactories.HIVE_BUILDER));
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, true, mdProvider.getMetadataProvider(), null,
-            new HiveFilterProjectTSTransposeRule(Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY,
-                    HiveProject.class, HiveRelFactories.HIVE_PROJECT_FACTORY, HiveTableScan.class));
+                HiveFilterProjectTSTransposeRule.INSTANCE, HiveFilterProjectTSTransposeRule.INSTANCE_DRUID,
+                HiveProjectFilterPullUpConstantsRule.INSTANCE);
 
         // 9.2.  Introduce exchange operators below join/multijoin operators
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
@@ -1222,7 +1222,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       rules.add(HiveFilterJoinRule.JOIN);
       rules.add(HiveFilterJoinRule.FILTER_ON_JOIN);
       rules.add(new HiveFilterAggregateTransposeRule(Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY, Aggregate.class));
-      rules.add(new FilterMergeRule(HiveRelFactories.HIVE_FILTER_FACTORY));
+      rules.add(new FilterMergeRule(HiveRelFactories.HIVE_BUILDER));
       if (conf.getBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_REDUCE_WITH_STATS)) {
         rules.add(HiveReduceExpressionsWithStatsRule.INSTANCE);
       }
@@ -1303,9 +1303,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // matches FIL-PROJ-TS
       perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
       basePlan = hepPlan(basePlan, true, mdProvider, null,
-          new HiveFilterProjectTSTransposeRule(
-              Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY, HiveProject.class,
-              HiveRelFactories.HIVE_PROJECT_FACTORY, TableScan.class),
+          HiveFilterProjectTSTransposeRule.INSTANCE, HiveFilterProjectTSTransposeRule.INSTANCE_DRUID,
           HiveProjectFilterPullUpConstantsRule.INSTANCE);
       perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER,
         "Calcite: Prejoin ordering transformation, Rerun PPD");
@@ -1822,8 +1820,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
           }
           List<Interval> intervals = Arrays.asList(DruidTable.DEFAULT_INTERVAL);
 
-          DruidTable druidTable = new DruidTable(new DruidSchema(address),
-                  dataSource, rowType, metrics, intervals, DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+          DruidTable druidTable = new DruidTable(new DruidSchema(address, address, false),
+                  dataSource, RelDataTypeImpl.proto(rowType), metrics, DruidTable.DEFAULT_TIMESTAMP_COLUMN, intervals);
           final TableScan scan = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
                   optTable, null == tableAlias ? tabMetaData.getTableName() : tableAlias,
                   getAliasId(tableAlias, qb), HiveConf.getBoolVar(conf,

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 82080eb..5e708d3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -73,8 +73,8 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBOMaxNumToCNF.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBOMaxNumToCNF.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBOMaxNumToCNF.java
index 277ac1e..cf72b7c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBOMaxNumToCNF.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBOMaxNumToCNF.java
@@ -23,6 +23,7 @@ import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.junit.Test;
@@ -49,7 +50,7 @@ public class TestCBOMaxNumToCNF {
                     rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
                             rexBuilder.makeInputRef(typeFactory.createSqlType(SqlTypeName.INTEGER), 1),
                             rexBuilder.makeLiteral(8, typeFactory.createSqlType(SqlTypeName.INTEGER), false))));
-    final RexNode newCond = HiveRexUtil.toCnf(rexBuilder, maxNumNodesCNF, cond);
+    final RexNode newCond = RexUtil.toCnf(rexBuilder, maxNumNodesCNF, cond);
 
     assertEquals(newCond.toString(), "AND(OR(=($0, 1), =($0, 0)), OR(=($0, 1), =($1, 8)))");
   }
@@ -75,7 +76,7 @@ public class TestCBOMaxNumToCNF {
                     rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
                             rexBuilder.makeInputRef(typeFactory.createSqlType(SqlTypeName.INTEGER), 1),
                             rexBuilder.makeLiteral(8, typeFactory.createSqlType(SqlTypeName.INTEGER), false))));
-    final RexNode newCond = HiveRexUtil.toCnf(rexBuilder, maxNumNodesCNF, cond);
+    final RexNode newCond = RexUtil.toCnf(rexBuilder, maxNumNodesCNF, cond);
 
     assertEquals(newCond.toString(), "OR(=($0, 1), =($0, 2), AND(=($0, 0), =($1, 8)))");
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
index 2830f1f..44e157b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
@@ -61,7 +61,7 @@ public class TestCBORuleFiredOnlyOnce {
 
     // Create rules registry to not trigger a rule more than once
     HiveRulesRegistry registry = new HiveRulesRegistry();
-    HivePlannerContext context = new HivePlannerContext(null, null, registry);
+    HivePlannerContext context = new HivePlannerContext(null, registry);
     HepPlanner planner = new HepPlanner(programBuilder.build(), context);
 
     // Cluster

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/druid_basic2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_basic2.q.out b/ql/src/test/results/clientpositive/druid_basic2.q.out
index 3205905..858f550 100644
--- a/ql/src/test/results/clientpositive/druid_basic2.q.out
+++ b/ql/src/test/results/clientpositive/druid_basic2.q.out
@@ -79,8 +79,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot"],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           GatherStats: false
           Select Operator
@@ -108,8 +108,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":["delta"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":[],"metrics":["delta"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           GatherStats: false
           Select Operator
@@ -139,8 +139,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           GatherStats: false
           Select Operator
@@ -170,8 +170,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"ALL","dimensions":["robot"],"limitSpec":{"type":"default"},"filter":{"type":"selector","dimension":"language","value":"en"},"aggregations":[{"type":"longSum","name":"dummy_agg","fieldName":"dummy_agg"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type groupBy
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"all","dimensions":["robot"],"limitSpec":{"type":"default"},"filter":{"type":"selector","dimension":"language","value":"en"},"aggregations":[{"type":"longSum","name":"dummy_agg","fieldName":"dummy_agg"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type GROUP_BY
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           GatherStats: false
           Select Operator
@@ -218,8 +218,8 @@ STAGE PLANS:
             alias: druid_table_1
             filterExpr: language is not null (type: boolean)
             properties:
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             GatherStats: false
             Filter Operator
@@ -243,8 +243,8 @@ STAGE PLANS:
             alias: druid_table_1
             filterExpr: language is not null (type: boolean)
             properties:
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             GatherStats: false
             Filter Operator
@@ -279,8 +279,8 @@ STAGE PLANS:
               columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
               columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
               druid.datasource wikipedia
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
 #### A masked pattern was here ####
               name default.druid_table_1
               numFiles 0
@@ -304,8 +304,8 @@ STAGE PLANS:
                 columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
                 columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
                 druid.datasource wikipedia
-                druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-                druid.query.type select
+                druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+                druid.query.type SELECT
 #### A masked pattern was here ####
                 name default.druid_table_1
                 numFiles 0
@@ -403,8 +403,8 @@ STAGE PLANS:
           TableScan
             alias: druid_table_1
             properties:
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
             GatherStats: false
             Select Operator
@@ -418,8 +418,8 @@ STAGE PLANS:
           TableScan
             alias: druid_table_1
             properties:
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             GatherStats: false
             Reduce Output Operator
@@ -445,8 +445,8 @@ STAGE PLANS:
               columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
               columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
               druid.datasource wikipedia
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
 #### A masked pattern was here ####
               name default.druid_table_1
               numFiles 0
@@ -470,8 +470,8 @@ STAGE PLANS:
                 columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
                 columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
                 druid.datasource wikipedia
-                druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-                druid.query.type select
+                druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+                druid.query.type SELECT
 #### A masked pattern was here ####
                 name default.druid_table_1
                 numFiles 0

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/druid_intervals.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_intervals.q.out b/ql/src/test/results/clientpositive/druid_intervals.q.out
index 984bb79..0cb373b 100644
--- a/ql/src/test/results/clientpositive/druid_intervals.q.out
+++ b/ql/src/test/results/clientpositive/druid_intervals.q.out
@@ -81,8 +81,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp)
@@ -113,8 +113,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/2012-03-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/2012-03-01T00:00:00.000Z"],"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp)
@@ -145,8 +145,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2012-03-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["2010-01-01T00:00:00.000Z/2012-03-01T00:00:00.001Z"],"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp)
@@ -179,8 +179,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["2010-01-01T00:00:00.000Z/2011-01-01T00:00:00.000Z"],"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp)
@@ -211,8 +211,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["2010-01-01T00:00:00.000Z/2011-01-01T00:00:00.001Z"],"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp)
@@ -245,8 +245,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00","2012-01-01T00:00:00.000-08:00/2013-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["2010-01-01T00:00:00.000Z/2011-01-01T00:00:00.001Z","2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.001Z"],"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp)
@@ -279,8 +279,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2012-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["2010-01-01T00:00:00.000Z/2012-01-01T00:00:00.001Z"],"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp)
@@ -311,8 +311,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2010-01-01T00:00:00.001-08:00","2011-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["2010-01-01T00:00:00.000Z/2010-01-01T00:00:00.001Z","2011-01-01T00:00:00.000Z/2011-01-01T00:00:00.001Z"],"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp)
@@ -341,8 +341,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2010-01-01T00:00:00.001-08:00","2011-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"filter":{"type":"selector","dimension":"robot","value":"user1"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-            druid.query.type select
+            druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["2010-01-01T00:00:00.000Z/2010-01-01T00:00:00.001Z","2011-01-01T00:00:00.000Z/2011-01-01T00:00:00.001Z"],"filter":{"type":"selector","dimension":"robot","value":"user1"},"dimensions":[],"metrics":[],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+            druid.query.type SELECT
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), 'user1' (type: string)
@@ -372,8 +372,8 @@ STAGE PLANS:
             alias: druid_table_1
             filterExpr: ((__time) IN ('2010-01-01 00:00:00', '2011-01-01 00:00:00') or (robot = 'user1')) (type: boolean)
             properties:
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: ((__time) IN ('2010-01-01 00:00:00', '2011-01-01 00:00:00') or (robot = 'user1')) (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/druid_timeseries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_timeseries.q.out b/ql/src/test/results/clientpositive/druid_timeseries.q.out
index 8d974a4..3708fba 100644
--- a/ql/src/test/results/clientpositive/druid_timeseries.q.out
+++ b/ql/src/test/results/clientpositive/druid_timeseries.q.out
@@ -81,8 +81,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"ALL","aggregations":[{"type":"longMax","name":"$f0","fieldName":"added"},{"type":"doubleSum","name":"$f1","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"all","aggregations":[{"type":"longMax","name":"$f0","fieldName":"added"},{"type":"doubleSum","name":"$f1","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: $f0 (type: bigint), $f1 (type: float)
@@ -113,8 +113,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"NONE","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"NONE","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -145,8 +145,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"YEAR","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"YEAR","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -177,8 +177,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"QUARTER","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"QUARTER","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -209,8 +209,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"MONTH","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"MONTH","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -241,8 +241,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"WEEK","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"WEEK","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -273,8 +273,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"DAY","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"DAY","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -305,8 +305,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"HOUR","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"HOUR","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -337,8 +337,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"MINUTE","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"MINUTE","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -369,8 +369,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"SECOND","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"SECOND","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -403,8 +403,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"HOUR","filter":{"type":"selector","dimension":"robot","value":"1"},"aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type timeseries
+            druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":false,"granularity":"HOUR","filter":{"type":"selector","dimension":"robot","value":"1"},"aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type TIMESERIES
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
@@ -442,8 +442,8 @@ STAGE PLANS:
             alias: druid_table_1
             filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
             properties:
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
@@ -521,8 +521,8 @@ STAGE PLANS:
             alias: druid_table_1
             filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
             properties:
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/druid_topn.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_topn.q.out b/ql/src/test/results/clientpositive/druid_topn.q.out
index 17bdaed..51f1fb5 100644
--- a/ql/src/test/results/clientpositive/druid_topn.q.out
+++ b/ql/src/test/results/clientpositive/druid_topn.q.out
@@ -87,8 +87,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"ALL","dimension":"robot","metric":"$f1","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":100}
-            druid.query.type topN
+            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"all","dimension":"robot","metric":"$f1","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"threshold":100}
+            druid.query.type TOP_N
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: robot (type: string), $f1 (type: bigint), $f2 (type: float)
@@ -123,8 +123,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"NONE","dimension":"robot","metric":"$f3","aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":100}
-            druid.query.type topN
+            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"NONE","dimension":"robot","metric":"$f3","aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"threshold":100}
+            druid.query.type TOP_N
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
@@ -159,8 +159,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"YEAR","dimension":"robot","metric":"$f3","aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":10}
-            druid.query.type topN
+            druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"YEAR","dimension":"robot","metric":"$f3","aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"threshold":10}
+            druid.query.type TOP_N
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
@@ -195,8 +195,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f3","direction":"ascending"}]},"aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type groupBy
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f3","direction":"ascending"}]},"aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type GROUP_BY
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
@@ -231,8 +231,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot","namespace"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f4","direction":"descending"},{"dimension":"$f3","direction":"descending"}]},"aggregations":[{"type":"longMax","name":"$f3","fieldName":"added"},{"type":"doubleSum","name":"$f4","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type groupBy
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot","namespace"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f4","direction":"descending"},{"dimension":"$f3","direction":"descending"}]},"aggregations":[{"type":"longMax","name":"$f3","fieldName":"added"},{"type":"doubleSum","name":"$f4","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type GROUP_BY
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: robot (type: string), __time (type: timestamp), $f3 (type: bigint), $f4 (type: float)
@@ -267,8 +267,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot","namespace"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"robot","direction":"ascending"},{"dimension":"$f3","direction":"descending"}]},"aggregations":[{"type":"longMax","name":"$f3","fieldName":"added"},{"type":"doubleSum","name":"$f4","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type groupBy
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot","namespace"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"robot","direction":"ascending"},{"dimension":"$f3","direction":"descending"}]},"aggregations":[{"type":"longMax","name":"$f3","fieldName":"added"},{"type":"doubleSum","name":"$f4","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type GROUP_BY
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: robot (type: string), __time (type: timestamp), $f3 (type: bigint), $f4 (type: float)
@@ -305,8 +305,8 @@ STAGE PLANS:
         TableScan
           alias: druid_table_1
           properties:
-            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"YEAR","dimensions":[],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f2","direction":"ascending"}]},"filter":{"type":"selector","dimension":"robot","value":"1"},"aggregations":[{"type":"longMax","name":"$f1_0","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
-            druid.query.type groupBy
+            druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"YEAR","dimensions":[],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f2","direction":"ascending"}]},"filter":{"type":"selector","dimension":"robot","value":"1"},"aggregations":[{"type":"longMax","name":"$f1_0","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"]}
+            druid.query.type GROUP_BY
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: '1' (type: string), __time (type: timestamp), $f1_0 (type: bigint), $f2 (type: float)
@@ -349,8 +349,8 @@ STAGE PLANS:
             alias: druid_table_1
             filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
             properties:
-              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
-              druid.query.type select
+              druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":false,"intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"all","pagingSpec":{"threshold":16384},"context":{"druid.query.fetch":false}}
+              druid.query.type SELECT
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
               predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)


[5/6] hive git commit: HIVE-13316: Upgrade to Calcite 1.10 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
deleted file mode 100644
index 43982aa..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
+++ /dev/null
@@ -1,1053 +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.hadoop.hive.ql.optimizer.calcite.druid;
-
-import java.io.IOException;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelFieldCollation.Direction;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Litmus;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
-import org.apache.hadoop.hive.conf.Constants;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
-import org.joda.time.Interval;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-
-/**
- * Relational expression representing a scan of a Druid data set.
- *
- * TODO: to be removed when Calcite is upgraded to 1.9
- */
-public class DruidQuery extends TableScan {
-
-  protected static final Logger LOG = LoggerFactory.getLogger(DruidQuery.class);
-
-  protected QuerySpec querySpec;
-
-  final DruidTable druidTable;
-  final List<Interval> intervals;
-  final ImmutableList<RelNode> rels;
-
-  private static final Pattern VALID_SIG = Pattern.compile("sf?p?a?l?");
-
-  /**
-   * Creates a DruidQuery.
-   *
-   * @param cluster        Cluster
-   * @param traitSet       Traits
-   * @param table          Table
-   * @param druidTable     Druid table
-   * @param interval       Interval for the query
-   * @param rels           Internal relational expressions
-   */
-  private DruidQuery(RelOptCluster cluster, RelTraitSet traitSet,
-      RelOptTable table, DruidTable druidTable,
-      List<Interval> intervals, List<RelNode> rels) {
-    super(cluster, traitSet, table);
-    this.druidTable = druidTable;
-    this.intervals = ImmutableList.copyOf(intervals);
-    this.rels = ImmutableList.copyOf(rels);
-
-    assert isValid(Litmus.THROW);
-  }
-
-  /** Returns a string describing the operations inside this query.
-   *
-   * <p>For example, "sfpal" means {@link TableScan} (s)
-   * followed by {@link Filter} (f)
-   * followed by {@link Project} (p)
-   * followed by {@link Aggregate} (a)
-   * followed by {@link Sort} (l).
-   *
-   * @see #isValidSignature(String)
-   */
-  String signature() {
-    final StringBuilder b = new StringBuilder();
-    for (RelNode rel : rels) {
-      b.append(rel instanceof TableScan ? 's'
-          : rel instanceof Project ? 'p'
-          : rel instanceof Filter ? 'f'
-          : rel instanceof Aggregate ? 'a'
-          : rel instanceof Sort ? 'l'
-          : '!');
-    }
-    return b.toString();
-  }
-
-  @Override public boolean isValid(Litmus litmus) {
-    if (!super.isValid(litmus)) {
-      return false;
-    }
-    final String signature = signature();
-    if (!isValidSignature(signature)) {
-      return litmus.fail("invalid signature");
-    }
-    if (rels.isEmpty()) {
-      return litmus.fail("must have at least one rel");
-    }
-    for (int i = 0; i < rels.size(); i++) {
-      final RelNode r = rels.get(i);
-      if (i == 0) {
-        if (!(r instanceof TableScan)) {
-          return litmus.fail("first rel must be TableScan");
-        }
-        if (r.getTable() != table) {
-          return litmus.fail("first rel must be based on table table");
-        }
-      } else {
-        final List<RelNode> inputs = r.getInputs();
-        if (inputs.size() != 1 || inputs.get(0) != rels.get(i - 1)) {
-          return litmus.fail("each rel must have a single input");
-        }
-        if (r instanceof Aggregate) {
-          final Aggregate aggregate = (Aggregate) r;
-          if (aggregate.getGroupSets().size() != 1
-              || aggregate.indicator) {
-            return litmus.fail("no grouping sets");
-          }
-          for (AggregateCall call : aggregate.getAggCallList()) {
-            if (call.filterArg >= 0) {
-              return litmus.fail("no filtered aggregate functions");
-            }
-          }
-        }
-        if (r instanceof Filter) {
-          final Filter filter = (Filter) r;
-          if (!isValidFilter(filter.getCondition())) {
-            return litmus.fail("invalid filter");
-          }
-        }
-        if (r instanceof Sort) {
-          final Sort sort = (Sort) r;
-          if (sort.offset != null && RexLiteral.intValue(sort.offset) != 0) {
-            return litmus.fail("offset not supported");
-          }
-        }
-      }
-    }
-    return true;
-  }
-
-  boolean isValidFilter(RexNode e) {
-    switch (e.getKind()) {
-    case INPUT_REF:
-    case LITERAL:
-      return true;
-    case AND:
-    case OR:
-    case NOT:
-    case EQUALS:
-    case LESS_THAN:
-    case LESS_THAN_OR_EQUAL:
-    case GREATER_THAN:
-    case GREATER_THAN_OR_EQUAL:
-    case BETWEEN:
-    case IN:
-    case CAST:
-      return areValidFilters(((RexCall) e).getOperands());
-    default:
-      return false;
-    }
-  }
-
-  private boolean areValidFilters(List<RexNode> es) {
-    for (RexNode e : es) {
-      if (!isValidFilter(e)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  /** Returns whether a signature represents an sequence of relational operators
-   * that can be translated into a valid Druid query. */
-  static boolean isValidSignature(String signature) {
-    return VALID_SIG.matcher(signature).matches();
-  }
-
-  /** Creates a DruidQuery. */
-  public static DruidQuery create(RelOptCluster cluster, RelTraitSet traitSet,
-      RelOptTable table, DruidTable druidTable, List<RelNode> rels) {
-    return new DruidQuery(cluster, traitSet, table, druidTable, druidTable.intervals, rels);
-  }
-
-  /** Creates a DruidQuery. */
-  private static DruidQuery create(RelOptCluster cluster, RelTraitSet traitSet,
-      RelOptTable table, DruidTable druidTable, List<Interval> intervals, List<RelNode> rels) {
-    return new DruidQuery(cluster, traitSet, table, druidTable, intervals, rels);
-  }
-
-  /** Extends a DruidQuery. */
-  public static DruidQuery extendQuery(DruidQuery query, RelNode r) {
-    final ImmutableList.Builder<RelNode> builder = ImmutableList.builder();
-    return DruidQuery.create(query.getCluster(), query.getTraitSet(), query.getTable(),
-            query.druidTable, query.intervals, builder.addAll(query.rels).add(r).build());
-  }
-
-  /** Extends a DruidQuery. */
-  public static DruidQuery extendQuery(DruidQuery query, List<Interval> intervals) {
-    return DruidQuery.create(query.getCluster(), query.getTraitSet(), query.getTable(),
-            query.druidTable, intervals, query.rels);
-  }
-
-  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    assert inputs.isEmpty();
-    return this;
-  }
-
-  @Override public RelDataType deriveRowType() {
-    return getCluster().getTypeFactory().createStructType(
-            Pair.right(Util.last(rels).getRowType().getFieldList()),
-            getQuerySpec().fieldNames);
-  }
-
-  public TableScan getTableScan() {
-    return (TableScan) rels.get(0);
-  }
-
-  public RelNode getTopNode() {
-    return Util.last(rels);
-  }
-
-  @Override public RelOptTable getTable() {
-    return table;
-  }
-
-  @Override public RelWriter explainTerms(RelWriter pw) {
-    for (RelNode rel : rels) {
-      if (rel instanceof TableScan) {
-        TableScan tableScan = (TableScan) rel;
-        pw.item("table", tableScan.getTable().getQualifiedName());
-        pw.item("intervals", intervals);
-      } else if (rel instanceof Filter) {
-        pw.item("filter", ((Filter) rel).getCondition());
-      } else if (rel instanceof Project) {
-        pw.item("projects", ((Project) rel).getProjects());
-      } else if (rel instanceof Aggregate) {
-        final Aggregate aggregate = (Aggregate) rel;
-        pw.item("groups", aggregate.getGroupSet())
-            .item("aggs", aggregate.getAggCallList());
-      } else if (rel instanceof Sort) {
-        final Sort sort = (Sort) rel;
-        for (Ord<RelFieldCollation> ord
-                : Ord.zip(sort.collation.getFieldCollations())) {
-          pw.item("sort" + ord.i, ord.e.getFieldIndex());
-        }
-        for (Ord<RelFieldCollation> ord
-            : Ord.zip(sort.collation.getFieldCollations())) {
-          pw.item("dir" + ord.i, ord.e.shortString());
-        }
-        pw.itemIf("fetch", sort.fetch, sort.fetch != null);
-      } else {
-        throw new AssertionError("rel type not supported in Druid query "
-            + rel);
-      }
-    }
-    return pw;
-  }
-
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
-      RelMetadataQuery mq) {
-    // Heuristic: we assume pushing query to Druid reduces cost by 90%
-    return Util.last(rels).computeSelfCost(planner, mq).multiplyBy(.1);
-  }
-
-  @Override public RelNode project(ImmutableBitSet fieldsUsed,
-      Set<RelDataTypeField> extraFields,
-      RelBuilder relBuilder) {
-    final int fieldCount = getRowType().getFieldCount();
-    if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))
-        && extraFields.isEmpty()) {
-      return this;
-    }
-    final List<RexNode> exprList = new ArrayList<>();
-    final List<String> nameList = new ArrayList<>();
-    final RexBuilder rexBuilder = getCluster().getRexBuilder();
-    final List<RelDataTypeField> fields = getRowType().getFieldList();
-
-    // Project the subset of fields.
-    for (int i : fieldsUsed) {
-      RelDataTypeField field = fields.get(i);
-      exprList.add(rexBuilder.makeInputRef(this, i));
-      nameList.add(field.getName());
-    }
-
-    // Project nulls for the extra fields. (Maybe a sub-class table has
-    // extra fields, but we don't.)
-    for (RelDataTypeField extraField : extraFields) {
-      exprList.add(
-          rexBuilder.ensureType(
-              extraField.getType(),
-              rexBuilder.constantNull(),
-              true));
-      nameList.add(extraField.getName());
-    }
-
-    HiveProject hp = (HiveProject) relBuilder.push(this).project(exprList, nameList).build();
-    hp.setSynthetic();
-    return hp;
-  }
-
-  public QuerySpec getQuerySpec() {
-    if (querySpec == null) {
-      querySpec = deriveQuerySpec();
-      assert querySpec != null : this;
-    }
-    return querySpec;
-  }
-
-  protected QuerySpec deriveQuerySpec() {
-    final RelDataType rowType = table.getRowType();
-    int i = 1;
-
-    RexNode filter = null;
-    if (i < rels.size() && rels.get(i) instanceof Filter) {
-      final Filter filterRel = (Filter) rels.get(i++);
-      filter = filterRel.getCondition();
-    }
-
-    List<RexNode> projects = null;
-    if (i < rels.size() && rels.get(i) instanceof Project) {
-      final Project project = (Project) rels.get(i++);
-      projects = project.getProjects();
-    }
-
-    ImmutableBitSet groupSet = null;
-    List<AggregateCall> aggCalls = null;
-    List<String> aggNames = null;
-    if (i < rels.size() && rels.get(i) instanceof Aggregate) {
-      final Aggregate aggregate = (Aggregate) rels.get(i++);
-      groupSet = aggregate.getGroupSet();
-      aggCalls = aggregate.getAggCallList();
-      aggNames = Util.skip(aggregate.getRowType().getFieldNames(),
-          groupSet.cardinality());
-    }
-
-    List<Integer> collationIndexes = null;
-    List<Direction> collationDirections = null;
-    Integer fetch = null;
-    if (i < rels.size() && rels.get(i) instanceof Sort) {
-      final Sort sort = (Sort) rels.get(i++);
-      collationIndexes = new ArrayList<>();
-      collationDirections = new ArrayList<>();
-      for (RelFieldCollation fCol: sort.collation.getFieldCollations()) {
-        collationIndexes.add(fCol.getFieldIndex());
-        collationDirections.add(fCol.getDirection());
-      }
-      fetch = sort.fetch != null ? RexLiteral.intValue(sort.fetch) : null;
-    }
-
-    if (i != rels.size()) {
-      throw new AssertionError("could not implement all rels");
-    }
-
-    return getQuery(rowType, filter, projects, groupSet, aggCalls, aggNames,
-            collationIndexes, collationDirections, fetch);
-  }
-
-  public String getQueryType() {
-    return getQuerySpec().queryType.getQueryName();
-  }
-
-  public String getQueryString() {
-    return getQuerySpec().queryString;
-  }
-
-  private QuerySpec getQuery(RelDataType rowType, RexNode filter, List<RexNode> projects,
-      ImmutableBitSet groupSet, List<AggregateCall> aggCalls, List<String> aggNames,
-      List<Integer> collationIndexes, List<Direction> collationDirections, Integer fetch) {
-    DruidQueryType queryType = DruidQueryType.SELECT;
-    final Translator translator = new Translator(druidTable, rowType);
-    List<String> fieldNames = rowType.getFieldNames();
-
-    // Handle filter
-    Json jsonFilter = null;
-    if (filter != null) {
-      jsonFilter = translator.translateFilter(filter);
-    }
-
-    // Then we handle project
-    if (projects != null) {
-      translator.metrics.clear();
-      translator.dimensions.clear();
-      final ImmutableList.Builder<String> builder = ImmutableList.builder();
-      for (RexNode project : projects) {
-        builder.add(translator.translate(project, true));
-      }
-      fieldNames = builder.build();
-    }
-
-    // Finally we handle aggregate and sort. Handling of these
-    // operators is more complex, since we need to extract
-    // the conditions to know whether the query will be
-    // executed as a Timeseries, TopN, or GroupBy in Druid
-    final List<String> dimensions = new ArrayList<>();
-    final List<JsonAggregation> aggregations = new ArrayList<>();
-    String granularity = "ALL";
-    Direction timeSeriesDirection = null;
-    JsonLimit limit = null;
-    if (groupSet != null) {
-      assert aggCalls != null;
-      assert aggNames != null;
-      assert aggCalls.size() == aggNames.size();
-
-      int timePositionIdx = -1;
-      final ImmutableList.Builder<String> builder = ImmutableList.builder();
-      if (projects != null) {
-        for (int groupKey : groupSet) {
-          final String s = fieldNames.get(groupKey);
-          final RexNode project = projects.get(groupKey);
-          if (project instanceof RexInputRef) {
-            // Reference, it could be to the timestamp column or any other dimension
-            final RexInputRef ref = (RexInputRef) project;
-            final String origin = druidTable.rowType.getFieldList().get(ref.getIndex()).getName();
-            if (origin.equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
-              granularity = "NONE";
-              builder.add(s);
-              assert timePositionIdx == -1;
-              timePositionIdx = groupKey;
-            } else {
-              dimensions.add(s);
-              builder.add(s);
-            }
-          } else if (project instanceof RexCall) {
-            // Call, check if we should infer granularity
-            RexCall call = (RexCall) project;
-            if (HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator())) {
-              granularity = call.getOperator().getName();
-              builder.add(s);
-              assert timePositionIdx == -1;
-              timePositionIdx = groupKey;
-            } else {
-              dimensions.add(s);
-              builder.add(s);
-            }
-          } else {
-            throw new AssertionError("incompatible project expression: " + project);
-          }
-        }
-      } else {
-        for (int groupKey : groupSet) {
-          final String s = fieldNames.get(groupKey);
-          if (s.equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
-            granularity = "NONE";
-            builder.add(s);
-            assert timePositionIdx == -1;
-            timePositionIdx = groupKey;
-          } else {
-            dimensions.add(s);
-            builder.add(s);
-          }
-        }
-      }
-
-      for (Pair<AggregateCall, String> agg : Pair.zip(aggCalls, aggNames)) {
-        final JsonAggregation jsonAggregation =
-            getJsonAggregation(fieldNames, agg.right, agg.left);
-        aggregations.add(jsonAggregation);
-        builder.add(jsonAggregation.name);
-      }
-
-      fieldNames = builder.build();
-
-      ImmutableList<JsonCollation> collations = null;
-      boolean sortsMetric = false;
-      if (collationIndexes != null) {
-        assert collationDirections != null;
-        ImmutableList.Builder<JsonCollation> colBuilder = new ImmutableList.Builder<JsonCollation>();
-        for (Pair<Integer,Direction> p : Pair.zip(collationIndexes, collationDirections)) {
-          colBuilder.add(new JsonCollation(fieldNames.get(p.left),
-                  p.right == Direction.DESCENDING ? "descending" : "ascending"));
-          if (p.left >= groupSet.cardinality() && p.right == Direction.DESCENDING) {
-            // Currently only support for DESC in TopN
-            sortsMetric = true;
-          } else if (p.left == timePositionIdx) {
-            assert timeSeriesDirection == null;
-            timeSeriesDirection = p.right;
-          }
-        }
-        collations = colBuilder.build();
-      }
-
-      limit = new JsonLimit("default", fetch, collations);
-
-      if (dimensions.isEmpty() && (collations == null || timeSeriesDirection != null)) {
-        queryType = DruidQueryType.TIMESERIES;
-        assert fetch == null;
-      } else if (dimensions.size() == 1 && sortsMetric && collations.size() == 1 && fetch != null) {
-        queryType = DruidQueryType.TOP_N;
-      } else {
-        queryType = DruidQueryType.GROUP_BY;
-      }
-    } else {
-      assert aggCalls == null;
-      assert aggNames == null;
-      assert collationIndexes == null || collationIndexes.isEmpty();
-      assert collationDirections == null || collationDirections.isEmpty();
-    }
-
-    final StringWriter sw = new StringWriter();
-    final JsonFactory factory = new JsonFactory();
-    try {
-      final JsonGenerator generator = factory.createGenerator(sw);
-
-      switch (queryType) {
-      case TIMESERIES:
-        generator.writeStartObject();
-
-        generator.writeStringField("queryType", "timeseries");
-        generator.writeStringField("dataSource", druidTable.dataSource);
-        generator.writeStringField("descending", timeSeriesDirection != null &&
-            timeSeriesDirection == Direction.DESCENDING ? "true" : "false");
-        generator.writeStringField("granularity", granularity);
-        writeFieldIf(generator, "filter", jsonFilter);
-        writeField(generator, "aggregations", aggregations);
-        writeFieldIf(generator, "postAggregations", null);
-        writeField(generator, "intervals", intervals);
-
-        generator.writeEndObject();
-        break;
-
-      case TOP_N:
-        generator.writeStartObject();
-
-        generator.writeStringField("queryType", "topN");
-        generator.writeStringField("dataSource", druidTable.dataSource);
-        generator.writeStringField("granularity", granularity);
-        generator.writeStringField("dimension", dimensions.get(0));
-        generator.writeStringField("metric", fieldNames.get(collationIndexes.get(0)));
-        writeFieldIf(generator, "filter", jsonFilter);
-        writeField(generator, "aggregations", aggregations);
-        writeFieldIf(generator, "postAggregations", null);
-        writeField(generator, "intervals", intervals);
-        generator.writeNumberField("threshold", fetch);
-
-        generator.writeEndObject();
-        break;
-
-      case GROUP_BY:
-        generator.writeStartObject();
-
-        if (aggregations.isEmpty()) {
-          // Druid requires at least one aggregation, otherwise gives:
-          //   Must have at least one AggregatorFactory
-          aggregations.add(
-              new JsonAggregation("longSum", "dummy_agg", "dummy_agg"));
-        }
-
-        generator.writeStringField("queryType", "groupBy");
-        generator.writeStringField("dataSource", druidTable.dataSource);
-        generator.writeStringField("granularity", granularity);
-        writeField(generator, "dimensions", dimensions);
-        writeFieldIf(generator, "limitSpec", limit);
-        writeFieldIf(generator, "filter", jsonFilter);
-        writeField(generator, "aggregations", aggregations);
-        writeFieldIf(generator, "postAggregations", null);
-        writeField(generator, "intervals", intervals);
-        writeFieldIf(generator, "having", null);
-
-        generator.writeEndObject();
-        break;
-
-      case SELECT:
-        generator.writeStartObject();
-
-        generator.writeStringField("queryType", "select");
-        generator.writeStringField("dataSource", druidTable.dataSource);
-        generator.writeStringField("descending", "false");
-        writeField(generator, "intervals", intervals);
-        writeFieldIf(generator, "filter", jsonFilter);
-        writeField(generator, "dimensions", translator.dimensions);
-        writeField(generator, "metrics", translator.metrics);
-        generator.writeStringField("granularity", granularity);
-
-        generator.writeFieldName("pagingSpec");
-        generator.writeStartObject();
-        generator.writeNumberField("threshold", fetch != null ? fetch : 1);
-        generator.writeEndObject();
-
-        generator.writeFieldName("context");
-        generator.writeStartObject();
-        generator.writeBooleanField(Constants.DRUID_QUERY_FETCH, fetch != null);
-        generator.writeEndObject();
-
-        generator.writeEndObject();
-        break;
-
-      default:
-        throw new AssertionError("unknown query type " + queryType);
-      }
-
-      generator.close();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-
-    return new QuerySpec(queryType, sw.toString(), fieldNames);
-  }
-
-  private JsonAggregation getJsonAggregation(List<String> fieldNames,
-      String name, AggregateCall aggCall) {
-    final List<String> list = new ArrayList<>();
-    for (Integer arg : aggCall.getArgList()) {
-      list.add(fieldNames.get(arg));
-    }
-    final String only = Iterables.getFirst(list, null);
-    final boolean b = aggCall.getType().getSqlTypeName() == SqlTypeName.DOUBLE;
-    switch (aggCall.getAggregation().getKind()) {
-    case COUNT:
-      if (aggCall.isDistinct()) {
-        return new JsonCardinalityAggregation("cardinality", name, list);
-      }
-      return new JsonAggregation("count", name, only);
-    case SUM:
-    case SUM0:
-      return new JsonAggregation(b ? "doubleSum" : "longSum", name, only);
-    case MIN:
-      return new JsonAggregation(b ? "doubleMin" : "longMin", name, only);
-    case MAX:
-      return new JsonAggregation(b ? "doubleMax" : "longMax", name, only);
-    default:
-      throw new AssertionError("unknown aggregate " + aggCall);
-    }
-  }
-
-  private static void writeField(JsonGenerator generator, String fieldName,
-      Object o) throws IOException {
-    generator.writeFieldName(fieldName);
-    writeObject(generator, o);
-  }
-
-  private static void writeFieldIf(JsonGenerator generator, String fieldName,
-      Object o) throws IOException {
-    if (o != null) {
-      writeField(generator, fieldName, o);
-    }
-  }
-
-  private static void writeArray(JsonGenerator generator, List<?> elements)
-      throws IOException {
-    generator.writeStartArray();
-    for (Object o : elements) {
-      writeObject(generator, o);
-    }
-    generator.writeEndArray();
-  }
-
-  private static void writeObject(JsonGenerator generator, Object o)
-      throws IOException {
-    if (o instanceof String) {
-      String s = (String) o;
-      generator.writeString(s);
-    } else if (o instanceof Interval) {
-      Interval i = (Interval) o;
-      generator.writeString(i.toString());
-    } else if (o instanceof Integer) {
-      Integer i = (Integer) o;
-      generator.writeNumber(i);
-    } else if (o instanceof List) {
-      writeArray(generator, (List<?>) o);
-    } else if (o instanceof Json) {
-      ((Json) o).write(generator);
-    } else {
-      throw new AssertionError("not a json object: " + o);
-    }
-  }
-
-  /** Druid query specification. */
-  public static class QuerySpec {
-    final DruidQueryType queryType;
-    final String queryString;
-    final List<String> fieldNames;
-
-    QuerySpec(DruidQueryType queryType, String queryString,
-        List<String> fieldNames) {
-      this.queryType = Preconditions.checkNotNull(queryType);
-      this.queryString = Preconditions.checkNotNull(queryString);
-      this.fieldNames = ImmutableList.copyOf(fieldNames);
-    }
-
-    @Override public int hashCode() {
-      return Objects.hash(queryType, queryString, fieldNames);
-    }
-
-    @Override public boolean equals(Object obj) {
-      return obj == this
-          || obj instanceof QuerySpec
-          && queryType == ((QuerySpec) obj).queryType
-          && queryString.equals(((QuerySpec) obj).queryString)
-          && fieldNames.equals(((QuerySpec) obj).fieldNames);
-    }
-
-    @Override public String toString() {
-      return "{queryType: " + queryType
-          + ", queryString: " + queryString
-          + ", fieldNames: " + fieldNames + "}";
-    }
-
-    String getQueryString(String pagingIdentifier, int offset) {
-      if (pagingIdentifier == null) {
-        return queryString;
-      }
-      return queryString.replace("\"threshold\":",
-          "\"pagingIdentifiers\":{\"" + pagingIdentifier + "\":" + offset
-              + "},\"threshold\":");
-    }
-  }
-
-  /** Translates scalar expressions to Druid field references. */
-  private static class Translator {
-    final List<String> dimensions = new ArrayList<>();
-    final List<String> metrics = new ArrayList<>();
-    final DruidTable druidTable;
-    final RelDataType rowType;
-
-    Translator(DruidTable druidTable, RelDataType rowType) {
-      this.druidTable = druidTable;
-      this.rowType = rowType;
-      for (RelDataTypeField f : rowType.getFieldList()) {
-        final String fieldName = f.getName();
-        if (druidTable.metricFieldNames.contains(fieldName)) {
-          metrics.add(fieldName);
-        } else if (!DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(fieldName)) {
-          dimensions.add(fieldName);
-        }
-      }
-    }
-
-    String translate(RexNode e, boolean set) {
-      switch (e.getKind()) {
-      case INPUT_REF:
-        final RexInputRef ref = (RexInputRef) e;
-        final String fieldName =
-            rowType.getFieldList().get(ref.getIndex()).getName();
-        if (set) {
-          if (druidTable.metricFieldNames.contains(fieldName)) {
-            metrics.add(fieldName);
-          } else if (!DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(fieldName)) {
-            dimensions.add(fieldName);
-          }
-        }
-        return fieldName;
-
-      case CAST:
-       return tr(e, 0, set);
-
-      case LITERAL:
-        return ((RexLiteral) e).getValue2().toString();
-
-      case OTHER_FUNCTION:
-        final RexCall call = (RexCall) e;
-        assert HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator());
-        return tr(call, 0, set);
-
-      default:
-        throw new AssertionError("invalid expression " + e);
-      }
-    }
-
-    @SuppressWarnings("incomplete-switch")
-    private JsonFilter translateFilter(RexNode e) {
-      RexCall call;
-      switch (e.getKind()) {
-      case EQUALS:
-      case NOT_EQUALS:
-      case GREATER_THAN:
-      case GREATER_THAN_OR_EQUAL:
-      case LESS_THAN:
-      case LESS_THAN_OR_EQUAL:
-        call = (RexCall) e;
-        int posRef;
-        int posConstant;
-        if (RexUtil.isConstant(call.getOperands().get(1))) {
-          posRef = 0;
-          posConstant = 1;
-        } else if (RexUtil.isConstant(call.getOperands().get(0))) {
-          posRef = 1;
-          posConstant = 0;
-        } else {
-          throw new AssertionError("it is not a valid comparison: " + e);
-        }
-        switch (e.getKind()) {
-        case EQUALS:
-          return new JsonSelector("selector", tr(e, posRef), tr(e, posConstant));
-        case NOT_EQUALS:
-          return new JsonCompositeFilter("not",
-              ImmutableList.of(new JsonSelector("selector", tr(e, posRef), tr(e, posConstant))));
-        case GREATER_THAN:
-          return new JsonBound("bound", tr(e, posRef), tr(e, posConstant), true, null, false,
-              false);
-        case GREATER_THAN_OR_EQUAL:
-          return new JsonBound("bound", tr(e, posRef), tr(e, posConstant), false, null, false,
-              false);
-        case LESS_THAN:
-          return new JsonBound("bound", tr(e, posRef), null, false, tr(e, posConstant), true,
-              false);
-        case LESS_THAN_OR_EQUAL:
-          return new JsonBound("bound", tr(e, posRef), null, false, tr(e, posConstant), false,
-              false);
-        }
-      case AND:
-      case OR:
-      case NOT:
-        call = (RexCall) e;
-        return new JsonCompositeFilter(e.getKind().toString().toLowerCase(),
-            translateFilters(call.getOperands()));
-      default:
-        throw new AssertionError("cannot translate filter: " + e);
-      }
-    }
-
-    private String tr(RexNode call, int index) {
-      return tr(call, index, false);
-    }
-
-    private String tr(RexNode call, int index, boolean set) {
-      return translate(((RexCall) call).getOperands().get(index), set);
-    }
-
-    private List<JsonFilter> translateFilters(List<RexNode> operands) {
-      final ImmutableList.Builder<JsonFilter> builder =
-          ImmutableList.builder();
-      for (RexNode operand : operands) {
-        builder.add(translateFilter(operand));
-      }
-      return builder.build();
-    }
-  }
-
-  /** Object that knows how to write itself to a
-   * {@link com.fasterxml.jackson.core.JsonGenerator}. */
-  private interface Json {
-    void write(JsonGenerator generator) throws IOException;
-  }
-
-  /** Aggregation element of a Druid "groupBy" or "topN" query. */
-  private static class JsonAggregation implements Json {
-    final String type;
-    final String name;
-    final String fieldName;
-
-    private JsonAggregation(String type, String name, String fieldName) {
-      this.type = type;
-      this.name = name;
-      this.fieldName = fieldName;
-    }
-
-    public void write(JsonGenerator generator) throws IOException {
-      generator.writeStartObject();
-      generator.writeStringField("type", type);
-      generator.writeStringField("name", name);
-      writeFieldIf(generator, "fieldName", fieldName);
-      generator.writeEndObject();
-    }
-  }
-
-  /** Collation element of a Druid "groupBy" query. */
-  private static class JsonLimit implements Json {
-    final String type;
-    final Integer limit;
-    final ImmutableList<JsonCollation> collations;
-
-    private JsonLimit(String type, Integer limit, ImmutableList<JsonCollation> collations) {
-      this.type = type;
-      this.limit = limit;
-      this.collations = collations;
-    }
-
-    public void write(JsonGenerator generator) throws IOException {
-      generator.writeStartObject();
-      generator.writeStringField("type", type);
-      writeFieldIf(generator, "limit", limit);
-      writeFieldIf(generator, "columns", collations);
-      generator.writeEndObject();
-    }
-  }
-
-  /** Collation element of a Druid "groupBy" query. */
-  private static class JsonCollation implements Json {
-    final String dimension;
-    final String direction;
-
-    private JsonCollation(String dimension, String direction) {
-      this.dimension = dimension;
-      this.direction = direction;
-    }
-
-    public void write(JsonGenerator generator) throws IOException {
-      generator.writeStartObject();
-      generator.writeStringField("dimension", dimension);
-      writeFieldIf(generator, "direction", direction);
-      generator.writeEndObject();
-    }
-  }
-
-  /** Aggregation element that calls the "cardinality" function. */
-  private static class JsonCardinalityAggregation extends JsonAggregation {
-    final List<String> fieldNames;
-
-    private JsonCardinalityAggregation(String type, String name,
-        List<String> fieldNames) {
-      super(type, name, null);
-      this.fieldNames = fieldNames;
-    }
-
-    public void write(JsonGenerator generator) throws IOException {
-      generator.writeStartObject();
-      generator.writeStringField("type", type);
-      generator.writeStringField("name", name);
-      writeFieldIf(generator, "fieldNames", fieldNames);
-      generator.writeEndObject();
-    }
-  }
-
-  /** Filter element of a Druid "groupBy" or "topN" query. */
-  private abstract static class JsonFilter implements Json {
-    final String type;
-
-    private JsonFilter(String type) {
-      this.type = type;
-    }
-  }
-
-  /** Equality filter. */
-  private static class JsonSelector extends JsonFilter {
-    private final String dimension;
-    private final String value;
-
-    private JsonSelector(String type, String dimension, String value) {
-      super(type);
-      this.dimension = dimension;
-      this.value = value;
-    }
-
-    public void write(JsonGenerator generator) throws IOException {
-      generator.writeStartObject();
-      generator.writeStringField("type", type);
-      generator.writeStringField("dimension", dimension);
-      generator.writeStringField("value", value);
-      generator.writeEndObject();
-    }
-  }
-
-  /** Bound filter. */
-  private static class JsonBound extends JsonFilter {
-    private final String dimension;
-    private final String lower;
-    private final boolean lowerStrict;
-    private final String upper;
-    private final boolean upperStrict;
-    private final boolean alphaNumeric;
-
-    private JsonBound(String type, String dimension, String lower,
-        boolean lowerStrict, String upper, boolean upperStrict,
-        boolean alphaNumeric) {
-      super(type);
-      this.dimension = dimension;
-      this.lower = lower;
-      this.lowerStrict = lowerStrict;
-      this.upper = upper;
-      this.upperStrict = upperStrict;
-      this.alphaNumeric = alphaNumeric;
-    }
-
-    public void write(JsonGenerator generator) throws IOException {
-      generator.writeStartObject();
-      generator.writeStringField("type", type);
-      generator.writeStringField("dimension", dimension);
-      if (lower != null) {
-        generator.writeStringField("lower", lower);
-        generator.writeBooleanField("lowerStrict", lowerStrict);
-      }
-      if (upper != null) {
-        generator.writeStringField("upper", upper);
-        generator.writeBooleanField("upperStrict", upperStrict);
-      }
-      generator.writeBooleanField("alphaNumeric", alphaNumeric);
-      generator.writeEndObject();
-    }
-  }
-
-  /** Filter that combines other filters using a boolean operator. */
-  private static class JsonCompositeFilter extends JsonFilter {
-    private final List<? extends JsonFilter> fields;
-
-    private JsonCompositeFilter(String type,
-        List<? extends JsonFilter> fields) {
-      super(type);
-      this.fields = fields;
-    }
-
-    public void write(JsonGenerator generator) throws IOException {
-      generator.writeStartObject();
-      generator.writeStringField("type", type);
-      switch (type) {
-      case "NOT":
-        writeField(generator, "field", fields.get(0));
-        break;
-      default:
-        writeField(generator, "fields", fields);
-      }
-      generator.writeEndObject();
-    }
-  }
-
-}
-
-// End DruidQuery.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
deleted file mode 100644
index 228b307..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
+++ /dev/null
@@ -1,42 +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.hadoop.hive.ql.optimizer.calcite.druid;
-
-/**
- * Type of Druid query.
- *
- * TODO: to be removed when Calcite is upgraded to 1.9
- */
-public enum DruidQueryType {
-  SELECT("select"),
-  TOP_N("topN"),
-  GROUP_BY("groupBy"),
-  TIMESERIES("timeseries");
-
-  private final String queryName;
-
-  private DruidQueryType(String queryName) {
-    this.queryName = queryName;
-  }
-
-  public String getQueryName() {
-    return this.queryName;
-  }
-}
-
-// End QueryType.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
deleted file mode 100644
index f68ffa5..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
+++ /dev/null
@@ -1,591 +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.hadoop.hive.ql.optimizer.calcite.druid;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rel.RelFieldCollation;
-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.Filter;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexShuttle;
-import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
-import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectSortTransposeRule;
-import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortProjectTransposeRule;
-import org.joda.time.Interval;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-
-/**
- * Rules and relational operators for {@link DruidQuery}.
- *
- * TODO: to be removed when Calcite is upgraded to 1.9
- */
-public class DruidRules {
-
-  protected static final Logger LOG = LoggerFactory.getLogger(DruidRules.class);
-
-  // Avoid instantiation
-  private DruidRules() {
-  }
-
-  public static final DruidFilterRule FILTER = new DruidFilterRule();
-  public static final DruidProjectRule PROJECT = new DruidProjectRule();
-  public static final DruidAggregateRule AGGREGATE = new DruidAggregateRule();
-  public static final DruidProjectAggregateRule PROJECT_AGGREGATE = new DruidProjectAggregateRule();
-  public static final DruidSortRule SORT = new DruidSortRule();
-  public static final DruidProjectSortRule PROJECT_SORT = new DruidProjectSortRule();
-  public static final DruidSortProjectRule SORT_PROJECT = new DruidSortProjectRule();
-
-  /** Predicate that returns whether Druid can not handle an aggregate. */
-  private static final Predicate<AggregateCall> BAD_AGG = new Predicate<AggregateCall>() {
-    public boolean apply(AggregateCall aggregateCall) {
-      switch (aggregateCall.getAggregation().getKind()) {
-        case COUNT:
-        case SUM:
-        case SUM0:
-        case MIN:
-        case MAX:
-          return false;
-        default:
-          return true;
-      }
-    }
-  };
-
-  /**
-   * Rule to push a {@link org.apache.calcite.rel.core.Filter} into a {@link DruidQuery}.
-   */
-  private static class DruidFilterRule extends RelOptRule {
-    private DruidFilterRule() {
-      super(operand(Filter.class,
-              operand(DruidQuery.class, none())));
-    }
-
-    public void onMatch(RelOptRuleCall call) {
-      final Filter filter = call.rel(0);
-      final DruidQuery query = call.rel(1);
-      if (!DruidQuery.isValidSignature(query.signature() + 'f')
-              || !query.isValidFilter(filter.getCondition())) {
-        return;
-      }
-      // Timestamp
-      int timestampFieldIdx = -1;
-      for (int i = 0; i < query.getRowType().getFieldCount(); i++) {
-        if (DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(
-                query.getRowType().getFieldList().get(i).getName())) {
-          timestampFieldIdx = i;
-          break;
-        }
-      }
-      final Pair<List<RexNode>, List<RexNode>> pair = splitFilters(
-              filter.getCluster().getRexBuilder(), query, filter.getCondition(), timestampFieldIdx);
-      if (pair == null) {
-        // We can't push anything useful to Druid.
-        return;
-      }
-      List<Interval> intervals = null;
-      if (!pair.left.isEmpty()) {
-        intervals = DruidIntervalUtils.createInterval(
-                query.getRowType().getFieldList().get(timestampFieldIdx).getType(),
-                pair.left);
-        if (intervals == null) {
-          // We can't push anything useful to Druid.
-          return;
-        }
-      }
-      DruidQuery newDruidQuery = query;
-      if (!pair.right.isEmpty()) {
-        if (!validConditions(pair.right)) {
-          return;
-        }
-        final RelNode newFilter = filter.copy(filter.getTraitSet(), Util.last(query.rels),
-                RexUtil.composeConjunction(filter.getCluster().getRexBuilder(), pair.right, false));
-        newDruidQuery = DruidQuery.extendQuery(query, newFilter);
-      }
-      if (intervals != null) {
-        newDruidQuery = DruidQuery.extendQuery(newDruidQuery, intervals);
-      }
-      call.transformTo(newDruidQuery);
-    }
-
-    /* Splits the filter condition in two groups: those that filter on the timestamp column
-     * and those that filter on other fields */
-    private static Pair<List<RexNode>, List<RexNode>> splitFilters(final RexBuilder rexBuilder,
-            final DruidQuery input, RexNode cond, final int timestampFieldIdx) {
-      final List<RexNode> timeRangeNodes = new ArrayList<>();
-      final List<RexNode> otherNodes = new ArrayList<>();
-      List<RexNode> conjs = RelOptUtil.conjunctions(cond);
-      if (conjs.isEmpty()) {
-        // We do not transform
-        return null;
-      }
-      // Number of columns with the dimensions and timestamp
-      int max = input.getRowType().getFieldCount() - input.druidTable.metricFieldNames.size();
-      for (RexNode conj : conjs) {
-        final RelOptUtil.InputReferencedVisitor visitor = new RelOptUtil.InputReferencedVisitor();
-        conj.accept(visitor);
-        if (visitor.inputPosReferenced.contains(timestampFieldIdx)) {
-          if (visitor.inputPosReferenced.size() != 1) {
-            // Complex predicate, transformation currently not supported
-            return null;
-          }
-          timeRangeNodes.add(conj);
-        } else if (!visitor.inputPosReferenced.tailSet(max).isEmpty()) {
-          // Filter on metrics, not supported in Druid
-          return null;
-        } else {
-          otherNodes.add(conj);
-        }
-      }
-      return Pair.of(timeRangeNodes, otherNodes);
-    }
-
-    /* Checks that all conditions are on ref + literal*/
-    private static boolean validConditions(List<RexNode> nodes) {
-      for (RexNode node: nodes) {
-        try {
-          node.accept(
-              new RexVisitorImpl<Void>(true) {
-                @SuppressWarnings("incomplete-switch")
-                @Override public Void visitCall(RexCall call) {
-                  switch (call.getKind()) {
-                    case CAST:
-                      // Only if on top of ref or literal
-                      if (call.getOperands().get(0) instanceof RexInputRef ||
-                              call.getOperands().get(0) instanceof RexLiteral) {
-                        break;
-                      }
-                      // Not supported
-                      throw Util.FoundOne.NULL;
-                    case EQUALS:
-                    case LESS_THAN:
-                    case LESS_THAN_OR_EQUAL:
-                    case GREATER_THAN:
-                    case GREATER_THAN_OR_EQUAL:
-                      // Check cast
-                      RexNode left = call.getOperands().get(0);
-                      if (left.getKind() == SqlKind.CAST) {
-                        left = ((RexCall)left).getOperands().get(0);
-                      }
-                      RexNode right = call.getOperands().get(1);
-                      if (right.getKind() == SqlKind.CAST) {
-                        right = ((RexCall)right).getOperands().get(0);
-                      }
-                      if (left instanceof RexInputRef && right instanceof RexLiteral) {
-                        break;
-                      }
-                      if (right instanceof RexInputRef && left instanceof RexLiteral) {
-                        break;
-                      }
-                      // Not supported if it is not ref + literal
-                      throw Util.FoundOne.NULL;
-                    case BETWEEN:
-                    case IN:
-                      // Not supported here yet
-                      throw Util.FoundOne.NULL;
-                  }
-                  return super.visitCall(call);
-                }
-              });
-        } catch (Util.FoundOne e) {
-          return false;
-        }
-      }
-      return true;
-    }
-  }
-
-  /**
-   * Rule to push a {@link org.apache.calcite.rel.core.Project} into a {@link DruidQuery}.
-   */
-  private static class DruidProjectRule extends RelOptRule {
-    private DruidProjectRule() {
-      super(operand(Project.class,
-              operand(DruidQuery.class, none())));
-    }
-
-    public void onMatch(RelOptRuleCall call) {
-      final Project project = call.rel(0);
-      final DruidQuery query = call.rel(1);
-      if (!DruidQuery.isValidSignature(query.signature() + 'p')) {
-        return;
-      }
-
-      if (canProjectAll(project.getProjects())) {
-        // All expressions can be pushed to Druid in their entirety.
-        final RelNode newProject = project.copy(project.getTraitSet(),
-                ImmutableList.of(Util.last(query.rels)));
-        RelNode newNode = DruidQuery.extendQuery(query, newProject);
-        call.transformTo(newNode);
-        return;
-      }
-      final Pair<List<RexNode>, List<RexNode>> pair = splitProjects(
-              project.getCluster().getRexBuilder(), query, project.getProjects());
-      if (pair == null) {
-        // We can't push anything useful to Druid.
-        return;
-      }
-      final List<RexNode> above = pair.left;
-      final List<RexNode> below = pair.right;
-      final RelDataTypeFactory.FieldInfoBuilder builder = project.getCluster().getTypeFactory()
-              .builder();
-      final RelNode input = Util.last(query.rels);
-      for (RexNode e : below) {
-        final String name;
-        if (e instanceof RexInputRef) {
-          name = input.getRowType().getFieldNames().get(((RexInputRef) e).getIndex());
-        } else {
-          name = null;
-        }
-        builder.add(name, e.getType());
-      }
-      final RelNode newProject = project.copy(project.getTraitSet(), input, below, builder.build());
-      final DruidQuery newQuery = DruidQuery.extendQuery(query, newProject);
-      final RelNode newProject2 = project.copy(project.getTraitSet(), newQuery, above,
-              project.getRowType());
-      call.transformTo(newProject2);
-    }
-
-    private static boolean canProjectAll(List<RexNode> nodes) {
-      for (RexNode e : nodes) {
-        if (!(e instanceof RexInputRef)) {
-          return false;
-        }
-      }
-      return true;
-    }
-
-    private static Pair<List<RexNode>, List<RexNode>> splitProjects(final RexBuilder rexBuilder,
-            final RelNode input, List<RexNode> nodes) {
-      final RelOptUtil.InputReferencedVisitor visitor = new RelOptUtil.InputReferencedVisitor();
-      for (RexNode node : nodes) {
-        node.accept(visitor);
-      }
-      if (visitor.inputPosReferenced.size() == input.getRowType().getFieldCount()) {
-        // All inputs are referenced
-        return null;
-      }
-      final List<RexNode> belowNodes = new ArrayList<>();
-      final List<RelDataType> belowTypes = new ArrayList<>();
-      final List<Integer> positions = Lists.newArrayList(visitor.inputPosReferenced);
-      for (int i : positions) {
-        final RexNode node = rexBuilder.makeInputRef(input, i);
-        belowNodes.add(node);
-        belowTypes.add(node.getType());
-      }
-      final List<RexNode> aboveNodes = new ArrayList<>();
-      for (RexNode node : nodes) {
-        aboveNodes.add(node.accept(new RexShuttle() {
-          @Override
-          public RexNode visitInputRef(RexInputRef ref) {
-            final int index = positions.indexOf(ref.getIndex());
-            return rexBuilder.makeInputRef(belowTypes.get(index), index);
-          }
-        }));
-      }
-      return Pair.of(aboveNodes, belowNodes);
-    }
-  }
-
-  /**
-   * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} into a {@link DruidQuery}.
-   */
-  private static class DruidAggregateRule extends RelOptRule {
-    private DruidAggregateRule() {
-      super(operand(Aggregate.class,
-              operand(DruidQuery.class, none())));
-    }
-
-    public void onMatch(RelOptRuleCall call) {
-      final Aggregate aggregate = call.rel(0);
-      final DruidQuery query = call.rel(1);
-      if (!DruidQuery.isValidSignature(query.signature() + 'a')) {
-        return;
-      }
-      if (aggregate.indicator
-              || aggregate.getGroupSets().size() != 1
-              || Iterables.any(aggregate.getAggCallList(), BAD_AGG)
-              || !validAggregate(aggregate, query)) {
-        return;
-      }
-      final RelNode newAggregate = aggregate.copy(aggregate.getTraitSet(),
-              ImmutableList.of(Util.last(query.rels)));
-      call.transformTo(DruidQuery.extendQuery(query, newAggregate));
-    }
-
-    /* Check whether agg functions reference timestamp */
-    private static boolean validAggregate(Aggregate aggregate, DruidQuery query) {
-      ImmutableBitSet.Builder builder = ImmutableBitSet.builder();
-      for (AggregateCall aggCall : aggregate.getAggCallList()) {
-        builder.addAll(aggCall.getArgList());
-      }
-      return !checkTimestampRefOnQuery(builder.build(), query.getTopNode());
-    }
-  }
-
-  /**
-   * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} and
-   * {@link org.apache.calcite.rel.core.Project} into a {@link DruidQuery}.
-   */
-  private static class DruidProjectAggregateRule extends RelOptRule {
-    private DruidProjectAggregateRule() {
-      super(operand(Aggregate.class,
-              operand(Project.class,
-                      operand(DruidQuery.class, none()))));
-    }
-
-    public void onMatch(RelOptRuleCall call) {
-      final Aggregate aggregate = call.rel(0);
-      final Project project = call.rel(1);
-      final DruidQuery query = call.rel(2);
-      if (!DruidQuery.isValidSignature(query.signature() + 'p' + 'a')) {
-        return;
-      }
-      int timestampIdx;
-      if ((timestampIdx = validProject(project, query)) == -1) {
-        return;
-      }
-      if (aggregate.indicator
-              || aggregate.getGroupSets().size() != 1
-              || Iterables.any(aggregate.getAggCallList(), BAD_AGG)
-              || !validAggregate(aggregate, timestampIdx)) {
-        return;
-      }
-
-      final RelNode newProject = project.copy(project.getTraitSet(),
-              ImmutableList.of(Util.last(query.rels)));
-      final DruidQuery projectDruidQuery = DruidQuery.extendQuery(query, newProject);
-      final RelNode newAggregate = aggregate.copy(aggregate.getTraitSet(),
-              ImmutableList.of(Util.last(projectDruidQuery.rels)));
-      call.transformTo(DruidQuery.extendQuery(projectDruidQuery, newAggregate));
-    }
-
-    /* To be a valid Project, we allow it to contain references, and a single call
-     * to an EXTRACT function on the timestamp column. Returns the reference to
-     * the timestamp, if any. */
-    private static int validProject(Project project, DruidQuery query) {
-      List<RexNode> nodes = project.getProjects();
-      int idxTimestamp = -1;
-      for (int i = 0; i < nodes.size(); i++) {
-        final RexNode e = nodes.get(i);
-        if (e instanceof RexCall) {
-          // It is a call, check that it is EXTRACT and follow-up conditions
-          final RexCall call = (RexCall) e;
-          if (!HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator())) {
-            return -1;
-          }
-          if (idxTimestamp != -1) {
-            // Already one usage of timestamp column
-            return -1;
-          }
-          if (!(call.getOperands().get(0) instanceof RexInputRef)) {
-            return -1;
-          }
-          final RexInputRef ref = (RexInputRef) call.getOperands().get(0);
-          if (!(checkTimestampRefOnQuery(ImmutableBitSet.of(ref.getIndex()), query.getTopNode()))) {
-            return -1;
-          }
-          idxTimestamp = i;
-          continue;
-        }
-        if (!(e instanceof RexInputRef)) {
-          // It needs to be a reference
-          return -1;
-        }
-        final RexInputRef ref = (RexInputRef) e;
-        if (checkTimestampRefOnQuery(ImmutableBitSet.of(ref.getIndex()), query.getTopNode())) {
-          if (idxTimestamp != -1) {
-            // Already one usage of timestamp column
-            return -1;
-          }
-          idxTimestamp = i;
-        }
-      }
-      return idxTimestamp;
-    }
-
-    private static boolean validAggregate(Aggregate aggregate, int idx) {
-      if (!aggregate.getGroupSet().get(idx)) {
-        return false;
-      }
-      for (AggregateCall aggCall : aggregate.getAggCallList()) {
-        if (aggCall.getArgList().contains(idx)) {
-          return false;
-        }
-      }
-      return true;
-    }
-  }
-
-  /**
-   * Rule to push an {@link org.apache.calcite.rel.core.Sort} through a
-   * {@link org.apache.calcite.rel.core.Project}. Useful to transform
-   * to complex Druid queries.
-   */
-  private static class DruidProjectSortRule extends HiveSortProjectTransposeRule {
-    private DruidProjectSortRule() {
-      super(operand(Sort.class,
-              operand(Project.class,
-                      operand(DruidQuery.class, none()))));
-    }
-
-    @Override
-    public boolean matches(RelOptRuleCall call) {
-      return true;
-    }
-
-  }
-
-  /**
-   * Rule to push back {@link org.apache.calcite.rel.core.Project} through a
-   * {@link org.apache.calcite.rel.core.Sort}. Useful if after pushing Sort,
-   * we could not push it inside DruidQuery.
-   */
-  private static class DruidSortProjectRule extends HiveProjectSortTransposeRule {
-    private DruidSortProjectRule() {
-      super(operand(Project.class,
-              operand(Sort.class,
-                      operand(DruidQuery.class, none()))));
-    }
-  }
-
-  /**
-   * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} into a {@link DruidQuery}.
-   */
-  private static class DruidSortRule extends RelOptRule {
-    private DruidSortRule() {
-      super(operand(Sort.class,
-              operand(DruidQuery.class, none())));
-    }
-
-    public void onMatch(RelOptRuleCall call) {
-      final Sort sort = call.rel(0);
-      final DruidQuery query = call.rel(1);
-      if (!DruidQuery.isValidSignature(query.signature() + 'l')) {
-        return;
-      }
-      // Either it is:
-      // - a sort without limit on the time column on top of
-      //     Agg operator (transformable to timeseries query), or
-      // - it is a sort w/o limit on columns that do not include
-      //     the time column on top of Agg operator, or
-      // - a simple limit on top of other operator than Agg
-      if (!validSortLimit(sort, query)) {
-        return;
-      }
-      final RelNode newSort = sort.copy(sort.getTraitSet(),
-              ImmutableList.of(Util.last(query.rels)));
-      call.transformTo(DruidQuery.extendQuery(query, newSort));
-    }
-
-    /* Check sort valid */
-    private static boolean validSortLimit(Sort sort, DruidQuery query) {
-      if (sort.offset != null && RexLiteral.intValue(sort.offset) != 0) {
-        // offset not supported by Druid
-        return false;
-      }
-      if (query.getTopNode() instanceof Aggregate) {
-        final Aggregate topAgg = (Aggregate) query.getTopNode();
-        final ImmutableBitSet.Builder positionsReferenced = ImmutableBitSet.builder();
-        int metricsRefs = 0;
-        for (RelFieldCollation col : sort.collation.getFieldCollations()) {
-          int idx = col.getFieldIndex();
-          if (idx >= topAgg.getGroupCount()) {
-            metricsRefs++;
-            continue;
-          }
-          positionsReferenced.set(topAgg.getGroupSet().nth(idx));
-        }
-        boolean refsTimestamp =
-                checkTimestampRefOnQuery(positionsReferenced.build(), topAgg.getInput());
-        if (refsTimestamp && metricsRefs != 0) {
-          return false;
-        }
-        return true;
-      }
-      // If it is going to be a Druid select operator, we push the limit iff
-      // 1) it does not contain a sort specification (required by Druid) and
-      // 2) limit is smaller than select threshold, as otherwise it might be
-      //   better to obtain some parallelization and let global limit
-      //   optimizer kick in
-      HiveDruidConf conf = sort.getCluster().getPlanner()
-              .getContext().unwrap(HiveDruidConf.class);
-      return HiveCalciteUtil.pureLimitRelNode(sort) &&
-              RexLiteral.intValue(sort.fetch) <= conf.getSelectThreshold();
-    }
-  }
-
-  /* Check if any of the references leads to the timestamp column */
-  private static boolean checkTimestampRefOnQuery(ImmutableBitSet set, RelNode top) {
-    if (top instanceof Project) {
-      ImmutableBitSet.Builder newSet = ImmutableBitSet.builder();
-      final Project project = (Project) top;
-      for (int index : set) {
-        RexNode node = project.getProjects().get(index);
-        if (node instanceof RexInputRef) {
-          newSet.set(((RexInputRef)node).getIndex());
-        } else if (node instanceof RexCall) {
-          RexCall call = (RexCall) node;
-          assert HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator());
-          newSet.set(((RexInputRef)call.getOperands().get(0)).getIndex());
-        }
-      }
-      top = project.getInput();
-      set = newSet.build();
-    }
-
-    // Check if any references the timestamp column
-    for (int index : set) {
-      if (DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(top.getRowType().getFieldNames().get(index))) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-}
-
-// End DruidRules.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
deleted file mode 100644
index 3b3f68a..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
+++ /dev/null
@@ -1,51 +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.hadoop.hive.ql.optimizer.calcite.druid;
-
-import java.util.Map;
-
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.AbstractSchema;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-
-/**
- * Schema mapped onto a Druid instance.
- *
- * TODO: to be removed when Calcite is upgraded to 1.9
- */
-public class DruidSchema extends AbstractSchema {
-  final String url;
-
-  /**
-   * Creates a Druid schema.
-   *
-   * @param url URL of query REST service
-   */
-  public DruidSchema(String url) {
-    this.url = Preconditions.checkNotNull(url);
-  }
-
-  @Override protected Map<String, Table> getTableMap() {
-    final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
-    return builder.build();
-  }
-}
-
-// End DruidSchema.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
deleted file mode 100644
index 7288291..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
+++ /dev/null
@@ -1,121 +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.hadoop.hive.ql.optimizer.calcite.druid;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.calcite.interpreter.BindableConvention;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.logical.LogicalTableScan;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.schema.TranslatableTable;
-import org.apache.calcite.schema.impl.AbstractTable;
-import org.joda.time.DateTime;
-import org.joda.time.Interval;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-/**
- * Table mapped onto a Druid table.
- *
- * TODO: to be removed when Calcite is upgraded to 1.9
- */
-public class DruidTable extends AbstractTable implements TranslatableTable {
-
-  public static final String DEFAULT_TIMESTAMP_COLUMN = "__time";
-  public static final Interval DEFAULT_INTERVAL = new Interval(
-          new DateTime("1900-01-01"),
-          new DateTime("3000-01-01")
-  );
-
-  final DruidSchema schema;
-  final String dataSource;
-  final RelDataType rowType;
-  final RelProtoDataType protoRowType;
-  final ImmutableSet<String> metricFieldNames;
-  final ImmutableList<Interval> intervals;
-  final String timestampFieldName;
-
-  /**
-   * Creates a Druid table.
-   *
-   * @param schema Druid schema that contains this table
-   * @param dataSource Druid data source name
-   * @param protoRowType Field names and types
-   * @param metricFieldNames Names of fields that are metrics
-   * @param interval Default interval if query does not constrain the time
-   * @param timestampFieldName Name of the column that contains the time
-   */
-  public DruidTable(DruidSchema schema, String dataSource,
-      RelProtoDataType protoRowType, Set<String> metricFieldNames,
-      List<Interval> intervals, String timestampFieldName) {
-    this.schema = Preconditions.checkNotNull(schema);
-    this.dataSource = Preconditions.checkNotNull(dataSource);
-    this.rowType = null;
-    this.protoRowType = protoRowType;
-    this.metricFieldNames = ImmutableSet.copyOf(metricFieldNames);
-    this.intervals = ImmutableList.copyOf(intervals);
-    this.timestampFieldName = Preconditions.checkNotNull(timestampFieldName);
-  }
-
-  public DruidTable(DruidSchema schema, String dataSource,
-      RelDataType rowType, Set<String> metricFieldNames,
-      List<Interval> intervals, String timestampFieldName) {
-    this.schema = Preconditions.checkNotNull(schema);
-    this.dataSource = Preconditions.checkNotNull(dataSource);
-    this.rowType = Preconditions.checkNotNull(rowType);
-    this.protoRowType = null;
-    this.metricFieldNames = ImmutableSet.copyOf(metricFieldNames);
-    this.intervals = ImmutableList.copyOf(intervals);
-    this.timestampFieldName = Preconditions.checkNotNull(timestampFieldName);
-  }
-
-  public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-    final RelDataType thisRowType;
-    if (rowType != null) {
-      thisRowType = rowType;
-    } else {
-      // Generate
-      thisRowType = protoRowType.apply(typeFactory);
-    }
-    final List<String> fieldNames = thisRowType.getFieldNames();
-    Preconditions.checkArgument(fieldNames.contains(timestampFieldName));
-    Preconditions.checkArgument(fieldNames.containsAll(metricFieldNames));
-    return thisRowType;
-  }
-
-  public RelNode toRel(RelOptTable.ToRelContext context,
-      RelOptTable relOptTable) {
-    final RelOptCluster cluster = context.getCluster();
-    final TableScan scan = LogicalTableScan.create(cluster, relOptTable);
-    return DruidQuery.create(cluster,
-        cluster.traitSetOf(BindableConvention.INSTANCE), relOptTable, this,
-        ImmutableList.<RelNode>of(scan));
-  }
-
-}
-
-// End DruidTable.java
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
deleted file mode 100644
index 0686dff..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
+++ /dev/null
@@ -1,33 +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.hadoop.hive.ql.optimizer.calcite.druid;
-
-public class HiveDruidConf {
-
-  private int selectThreshold;
-
-
-  public HiveDruidConf(int selectThreshold) {
-    this.selectThreshold = selectThreshold;
-  }
-
-  public int getSelectThreshold() {
-    return selectThreshold;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java
index dc6b152..6df6026 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAggregate.java
@@ -35,7 +35,6 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.IntList;
 import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
 
 import com.google.common.collect.Sets;
@@ -90,7 +89,7 @@ public class HiveAggregate extends Aggregate implements HiveRelNode {
       final RelDataType inputRowType, boolean indicator,
       ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets,
       final List<AggregateCall> aggCalls) {
-    final IntList groupList = groupSet.toList();
+    final List<Integer> groupList = groupSet.asList();
     assert groupList.size() == groupSet.cardinality();
     final RelDataTypeFactory.FieldInfoBuilder builder = typeFactory.builder();
     final List<RelDataTypeField> fieldList = inputRowType.getFieldList();

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
deleted file mode 100644
index b3f8d9b..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
+++ /dev/null
@@ -1,54 +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.hadoop.hive.ql.optimizer.calcite.reloperators;
-
-import java.util.Set;
-
-import org.apache.calcite.sql.SqlFunction;
-import org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.ReturnTypes;
-
-import com.google.common.collect.Sets;
-
-public class HiveDateGranularity extends SqlFunction {
-
-  public static final SqlFunction YEAR = new HiveDateGranularity("YEAR");
-  public static final SqlFunction QUARTER = new HiveDateGranularity("QUARTER");
-  public static final SqlFunction MONTH = new HiveDateGranularity("MONTH");
-  public static final SqlFunction WEEK = new HiveDateGranularity("WEEK");
-  public static final SqlFunction DAY = new HiveDateGranularity("DAY");
-  public static final SqlFunction HOUR = new HiveDateGranularity("HOUR");
-  public static final SqlFunction MINUTE = new HiveDateGranularity("MINUTE");
-  public static final SqlFunction SECOND = new HiveDateGranularity("SECOND");
-
-  public static final Set<SqlFunction> ALL_FUNCTIONS =
-          Sets.newHashSet(YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE, SECOND);
-
-  private HiveDateGranularity(String name) {
-    super(
-        name,
-        SqlKind.OTHER_FUNCTION,
-        ReturnTypes.TIME_NULLABLE,
-        null,
-        OperandTypes.ANY,
-        SqlFunctionCategory.TIMEDATE);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveExtractDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveExtractDate.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveExtractDate.java
new file mode 100644
index 0000000..4edc4df
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveExtractDate.java
@@ -0,0 +1,50 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.reloperators;
+
+import java.util.Set;
+
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+
+import com.google.common.collect.Sets;
+
+public class HiveExtractDate extends SqlFunction {
+
+  public static final SqlFunction YEAR = new HiveExtractDate("YEAR");
+  public static final SqlFunction QUARTER = new HiveExtractDate("QUARTER");
+  public static final SqlFunction MONTH = new HiveExtractDate("MONTH");
+  public static final SqlFunction WEEK = new HiveExtractDate("WEEK");
+  public static final SqlFunction DAY = new HiveExtractDate("DAY");
+  public static final SqlFunction HOUR = new HiveExtractDate("HOUR");
+  public static final SqlFunction MINUTE = new HiveExtractDate("MINUTE");
+  public static final SqlFunction SECOND = new HiveExtractDate("SECOND");
+
+  public static final Set<SqlFunction> ALL_FUNCTIONS =
+          Sets.newHashSet(YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE, SECOND);
+
+  private HiveExtractDate(String name) {
+    super(name, SqlKind.EXTRACT, ReturnTypes.INTEGER_NULLABLE, null,
+            OperandTypes.INTERVALINTERVAL_INTERVALDATETIME,
+            SqlFunctionCategory.SYSTEM);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFloorDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFloorDate.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFloorDate.java
new file mode 100644
index 0000000..3d104ef
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveFloorDate.java
@@ -0,0 +1,64 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.reloperators;
+
+import java.util.Set;
+
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.fun.SqlMonotonicUnaryFunction;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+
+import com.google.common.collect.Sets;
+
+public class HiveFloorDate extends SqlMonotonicUnaryFunction {
+
+  public static final SqlFunction YEAR = new HiveFloorDate("FLOOR_YEAR");
+  public static final SqlFunction QUARTER = new HiveFloorDate("FLOOR_QUARTER");
+  public static final SqlFunction MONTH = new HiveFloorDate("FLOOR_MONTH");
+  public static final SqlFunction WEEK = new HiveFloorDate("FLOOR_WEEK");
+  public static final SqlFunction DAY = new HiveFloorDate("FLOOR_DAY");
+  public static final SqlFunction HOUR = new HiveFloorDate("FLOOR_HOUR");
+  public static final SqlFunction MINUTE = new HiveFloorDate("FLOOR_MINUTE");
+  public static final SqlFunction SECOND = new HiveFloorDate("FLOOR_SECOND");
+
+  public static final Set<SqlFunction> ALL_FUNCTIONS =
+          Sets.newHashSet(YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE, SECOND);
+
+  private HiveFloorDate(String name) {
+    super(name, SqlKind.FLOOR, ReturnTypes.ARG0_OR_EXACT_NO_SCALE, null,
+        OperandTypes.sequence(
+            "'" + SqlKind.FLOOR + "(<DATE> TO <TIME_UNIT>)'\n"
+            + "'" + SqlKind.FLOOR + "(<TIME> TO <TIME_UNIT>)'\n"
+            + "'" + SqlKind.FLOOR + "(<TIMESTAMP> TO <TIME_UNIT>)'",
+            OperandTypes.DATETIME,
+            OperandTypes.ANY),
+        SqlFunctionCategory.NUMERIC);
+  }
+
+  @Override
+  public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
+    // Monotonic iff its first argument is, but not strict.
+    return call.getOperandMonotonicity(0).unstrict();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
index e9a4d88..87e755c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateJoinTransposeRule.java
@@ -133,9 +133,10 @@ public class HiveAggregateJoinTransposeRule extends AggregateJoinTransposeRule {
     // Split join condition
     final List<Integer> leftKeys = Lists.newArrayList();
     final List<Integer> rightKeys = Lists.newArrayList();
+    final List<Boolean> filterNulls = Lists.newArrayList();
     RexNode nonEquiConj =
         RelOptUtil.splitJoinCondition(join.getLeft(), join.getRight(),
-            join.getCondition(), leftKeys, rightKeys);
+            join.getCondition(), leftKeys, rightKeys, filterNulls);
     // If it contains non-equi join conditions, we bail out
     if (!nonEquiConj.isAlwaysTrue()) {
       return;
@@ -271,7 +272,8 @@ public class HiveAggregateJoinTransposeRule extends AggregateJoinTransposeRule {
       RelOptUtil.areRowTypesEqual(r.getRowType(), aggregate.getRowType(), false)) {
       // no need to aggregate
     } else {
-      r = RelOptUtil.createProject(r, projects, null, true, projectFactory);
+      r = RelOptUtil.createProject(r, projects, null, true,
+              relBuilderFactory.create(aggregate.getCluster(), null));
       if (allColumnsInAggregate) {
         // let's see if we can convert
         List<RexNode> projects2 = new ArrayList<>();
@@ -290,7 +292,8 @@ public class HiveAggregateJoinTransposeRule extends AggregateJoinTransposeRule {
         if (projects2.size()
             == aggregate.getGroupSet().cardinality() + newAggCalls.size()) {
           // We successfully converted agg calls into projects.
-          r = RelOptUtil.createProject(r, projects2, null, true, projectFactory);
+          r = RelOptUtil.createProject(r, projects2, null, true,
+                  relBuilderFactory.create(aggregate.getCluster(), null));
           break b;
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
index 8af8a0d..c243266 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregateProjectMergeRule.java
@@ -141,7 +141,8 @@ public class HiveAggregateProjectMergeRule extends RelOptRule {
            i < newAggregate.getRowType().getFieldCount(); i++) {
         posList.add(i);
       }
-      rel = HiveRelOptUtil.createProject(HiveRelFactories.HIVE_PROJECT_FACTORY,
+      rel = HiveRelOptUtil.createProject(
+          HiveRelFactories.HIVE_BUILDER.create(aggregate.getCluster(), null),
           rel, posList);
 
     }


[6/6] hive git commit: HIVE-13316: Upgrade to Calcite 1.10 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
HIVE-13316: Upgrade to Calcite 1.10 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: b597ab2a07034b9c82e4bb0591123c3a115f27eb
Parents: 4b7f373
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Sep 28 20:23:33 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Tue Oct 18 12:27:41 2016 +0100

----------------------------------------------------------------------
 .../druid/HiveDruidQueryBasedInputFormat.java   |    6 +-
 .../serde/DruidGroupByQueryRecordReader.java    |    2 +-
 .../serde/DruidSelectQueryRecordReader.java     |    2 +-
 .../hadoop/hive/druid/serde/DruidSerDe.java     |    2 +-
 .../serde/DruidTimeseriesQueryRecordReader.java |    2 +-
 .../druid/serde/DruidTopNQueryRecordReader.java |    2 +-
 pom.xml                                         |    3 +-
 ql/pom.xml                                      |    7 +-
 .../calcite/HiveDefaultRelMetadataProvider.java |    2 +-
 .../optimizer/calcite/HivePlannerContext.java   |    9 +-
 .../ql/optimizer/calcite/HiveRelBuilder.java    |   18 +-
 .../ql/optimizer/calcite/HiveRelOptUtil.java    |    8 +-
 .../hive/ql/optimizer/calcite/HiveRexUtil.java  |  821 --------------
 .../optimizer/calcite/HiveTypeSystemImpl.java   |   39 +-
 .../calcite/cost/HiveDefaultCostModel.java      |    7 +-
 .../optimizer/calcite/cost/HiveRelMdCost.java   |   10 +-
 .../calcite/druid/DruidIntervalUtils.java       |  466 --------
 .../ql/optimizer/calcite/druid/DruidQuery.java  | 1053 ------------------
 .../optimizer/calcite/druid/DruidQueryType.java |   42 -
 .../ql/optimizer/calcite/druid/DruidRules.java  |  591 ----------
 .../ql/optimizer/calcite/druid/DruidSchema.java |   51 -
 .../ql/optimizer/calcite/druid/DruidTable.java  |  121 --
 .../optimizer/calcite/druid/HiveDruidConf.java  |   33 -
 .../calcite/reloperators/HiveAggregate.java     |    3 +-
 .../reloperators/HiveDateGranularity.java       |   54 -
 .../calcite/reloperators/HiveExtractDate.java   |   50 +
 .../calcite/reloperators/HiveFloorDate.java     |   64 ++
 .../rules/HiveAggregateJoinTransposeRule.java   |    9 +-
 .../rules/HiveAggregateProjectMergeRule.java    |    3 +-
 .../rules/HiveFilterProjectTSTransposeRule.java |   16 +-
 .../rules/HiveFilterProjectTransposeRule.java   |   21 +-
 .../calcite/rules/HivePreFilteringRule.java     |    7 +-
 .../rules/HiveReduceExpressionsRule.java        |  914 ++-------------
 .../HiveReduceExpressionsWithStatsRule.java     |    5 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      |  243 +---
 .../calcite/stats/HiveRelMdCollation.java       |   10 +-
 .../calcite/stats/HiveRelMdDistribution.java    |   10 +-
 .../calcite/stats/HiveRelMdPredicates.java      |   31 +-
 .../calcite/stats/HiveRelMdSelectivity.java     |   28 +-
 .../optimizer/calcite/stats/HiveRelMdSize.java  |   13 +-
 .../calcite/stats/HiveRelMdUniqueKeys.java      |   72 +-
 .../calcite/translator/ASTBuilder.java          |   49 +-
 .../calcite/translator/ASTConverter.java        |   51 +-
 .../calcite/translator/ExprNodeConverter.java   |   49 +-
 .../translator/PlanModifierForASTConv.java      |    5 +
 .../calcite/translator/RexNodeConverter.java    |   61 +-
 .../translator/SqlFunctionConverter.java        |   37 +-
 .../calcite/translator/TypeConverter.java       |   41 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   40 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |    2 +-
 .../optimizer/calcite/TestCBOMaxNumToCNF.java   |    5 +-
 .../calcite/TestCBORuleFiredOnlyOnce.java       |    2 +-
 .../results/clientpositive/druid_basic2.q.out   |   48 +-
 .../clientpositive/druid_intervals.q.out        |   40 +-
 .../clientpositive/druid_timeseries.q.out       |   52 +-
 .../results/clientpositive/druid_topn.q.out     |   32 +-
 .../clientpositive/explain_logical.q.out        |   48 +-
 .../clientpositive/groupby_sort_1_23.q.out      |   40 +-
 .../clientpositive/groupby_sort_skew_1_23.q.out |   40 +-
 .../results/clientpositive/limit_pushdown.q.out |   12 +-
 .../clientpositive/limit_pushdown3.q.out        |   12 +-
 .../clientpositive/llap/explainuser_4.q.out     |   32 +-
 .../clientpositive/llap/limit_pushdown.q.out    |    9 +-
 .../results/clientpositive/llap/lineage3.q.out  |    2 +-
 .../llap/table_access_keys_stats.q.out          |    6 +-
 .../llap/tez_dynpart_hashjoin_1.q.out           |   42 +-
 .../llap/tez_vector_dynpart_hashjoin_1.q.out    |   42 +-
 .../offset_limit_ppd_optimizer.q.out            |   12 +-
 .../results/clientpositive/perf/query75.q.out   |   12 +-
 .../spark/groupby_sort_1_23.q.out               |   32 +-
 .../spark/groupby_sort_skew_1_23.q.out          |   32 +-
 .../clientpositive/spark/limit_pushdown.q.out   |    9 +-
 .../spark/table_access_keys_stats.q.out         |    6 +-
 .../clientpositive/tez/explainanalyze_4.q.out   |   32 +-
 .../tez/vectorization_limit.q.out               |    9 +-
 .../clientpositive/vectorization_limit.q.out    |   12 +-
 76 files changed, 1136 insertions(+), 4669 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
index 3df1452..a18e590 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
@@ -25,6 +25,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.calcite.adapter.druid.DruidDateTimeUtils;
+import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -37,8 +39,6 @@ import org.apache.hadoop.hive.druid.serde.DruidSelectQueryRecordReader;
 import org.apache.hadoop.hive.druid.serde.DruidTimeseriesQueryRecordReader;
 import org.apache.hadoop.hive.druid.serde.DruidTopNQueryRecordReader;
 import org.apache.hadoop.hive.druid.serde.DruidWritable;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidIntervalUtils;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapred.JobConf;
@@ -273,7 +273,7 @@ public class HiveDruidQueryBasedInputFormat extends InputFormat<NullWritable, Dr
   }
 
   private static List<List<Interval>> createSplitsIntervals(List<Interval> intervals, int numSplits) {
-    final long totalTime = DruidIntervalUtils.extractTotalTime(intervals);
+    final long totalTime = DruidDateTimeUtils.extractTotalTime(intervals);
     long startTime = intervals.get(0).getStartMillis();
     long endTime = startTime;
     long currTime = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
index 226060f..49e096b 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
@@ -21,9 +21,9 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
 
+import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
index 70b493c..fccf7c4 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
@@ -22,8 +22,8 @@ import java.io.InputStream;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
 import org.apache.hadoop.io.NullWritable;
 
 import com.fasterxml.jackson.core.type.TypeReference;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
index 8f53d4a..238f7a3 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
@@ -25,11 +25,11 @@ import java.util.List;
 import java.util.Map.Entry;
 import java.util.Properties;
 
+import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.SerDeSpec;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
index 812ae03..b91178c 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
@@ -21,8 +21,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.List;
 
+import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
 import org.apache.hadoop.io.NullWritable;
 
 import com.fasterxml.jackson.core.type.TypeReference;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
index 0b87976..0b77a9b 100644
--- a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
@@ -22,8 +22,8 @@ import java.io.InputStream;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.calcite.adapter.druid.DruidTable;
 import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
 import org.apache.hadoop.io.NullWritable;
 
 import com.fasterxml.jackson.core.type.TypeReference;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5d13344..98d2dc2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -112,9 +112,10 @@
     <antlr.version>3.4</antlr.version>
     <apache-directory-server.version>1.5.6</apache-directory-server.version>
     <apache-directory-clientapi.version>0.1</apache-directory-clientapi.version>
+    <avatica.version>1.8.0</avatica.version>
     <avro.version>1.7.7</avro.version>
     <bonecp.version>0.8.0.RELEASE</bonecp.version>
-    <calcite.version>1.6.0</calcite.version>
+    <calcite.version>1.10.0</calcite.version>
     <datanucleus-api-jdo.version>4.2.1</datanucleus-api-jdo.version>
     <datanucleus-core.version>4.1.6</datanucleus-core.version>
     <datanucleus-rdbms.version>4.1.7</datanucleus-rdbms.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/pom.xml
----------------------------------------------------------------------
diff --git a/ql/pom.xml b/ql/pom.xml
index 2a93bb7..489c6f3 100644
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@ -383,8 +383,13 @@
     </dependency>
     <dependency>
       <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-avatica</artifactId>
+      <artifactId>calcite-druid</artifactId>
       <version>${calcite.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite.avatica</groupId>
+      <artifactId>avatica</artifactId>
+      <version>${avatica.version}</version>
       <exclusions>
         <!-- hsqldb interferes with the use of derby as the default db
           in hive's use of datanucleus.

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java
index c0609d7..75fb916 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveDefaultRelMetadataProvider.java
@@ -77,7 +77,7 @@ public class HiveDefaultRelMetadataProvider {
                     HiveRelMdDistribution.SOURCE,
                     HiveRelMdCollation.SOURCE,
                     HiveRelMdPredicates.SOURCE,
-                    new DefaultRelMetadataProvider()));
+                    DefaultRelMetadataProvider.INSTANCE));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
index 890aea1..8beb0dd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
@@ -19,19 +19,15 @@ package org.apache.hadoop.hive.ql.optimizer.calcite;
 
 import org.apache.calcite.plan.Context;
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveAlgorithmsConf;
-import org.apache.hadoop.hive.ql.optimizer.calcite.druid.HiveDruidConf;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRulesRegistry;
 
 
 public class HivePlannerContext implements Context {
   private HiveAlgorithmsConf algoConfig;
-  private HiveDruidConf druidConf;
   private HiveRulesRegistry registry;
 
-  public HivePlannerContext(HiveAlgorithmsConf algoConfig, HiveDruidConf druidConf,
-          HiveRulesRegistry registry) {
+  public HivePlannerContext(HiveAlgorithmsConf algoConfig, HiveRulesRegistry registry) {
     this.algoConfig = algoConfig;
-    this.druidConf = druidConf;
     this.registry = registry;
   }
 
@@ -39,9 +35,6 @@ public class HivePlannerContext implements Context {
     if (clazz.isInstance(algoConfig)) {
       return clazz.cast(algoConfig);
     }
-    if (clazz.isInstance(druidConf)) {
-      return clazz.cast(druidConf);
-    }
     if (clazz.isInstance(registry)) {
       return clazz.cast(registry);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
index 1c64d64..bc160d8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
@@ -81,7 +81,7 @@ public class HiveRelBuilder extends RelBuilder {
 
   @Override
   public RelBuilder filter(Iterable<? extends RexNode> predicates) {
-    final RexNode x = HiveRexUtil.simplify(cluster.getRexBuilder(),
+    final RexNode x = RexUtil.simplify(cluster.getRexBuilder(),
             RexUtil.composeConjunction(cluster.getRexBuilder(), predicates, false));
     if (!x.isAlwaysTrue()) {
       final RelNode input = build();
@@ -91,4 +91,20 @@ public class HiveRelBuilder extends RelBuilder {
     return this;
   }
 
+  /**
+   * Empty relationship can be expressed in many different ways, e.g.,
+   * filter(cond=false), empty LogicalValues(), etc. Calcite default implementation
+   * uses empty LogicalValues(); however, currently there is not an equivalent to
+   * this expression in Hive. Thus, we use limit 0, since Hive already includes
+   * optimizations that will do early pruning of the result tree when it is found,
+   * e.g., GlobalLimitOptimizer.
+   */
+  @Override
+  public RelBuilder empty() {
+    final RelNode input = build();
+    final RelNode sort = HiveRelFactories.HIVE_SORT_FACTORY.createSort(
+            input, RelCollations.of(), null, literal(0));
+    return this.push(sort);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
index 4c154d0..50fbb78 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
@@ -24,7 +24,6 @@ import java.util.List;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
@@ -34,6 +33,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
@@ -314,12 +314,12 @@ public class HiveRelOptUtil extends RelOptUtil {
    *
    * <p>Optimizes if the fields are the identity projection.
    *
-   * @param factory ProjectFactory
+   * @param relBuilder RelBuilder
    * @param child Input relational expression
    * @param posList Source of each projected field
    * @return Relational expression that projects given fields
    */
-  public static RelNode createProject(final RelFactories.ProjectFactory factory,
+  public static RelNode createProject(final RelBuilder relBuilder,
       final RelNode child, final List<Integer> posList) {
     RelDataType rowType = child.getRowType();
     final List<String> fieldNames = rowType.getFieldNames();
@@ -344,7 +344,7 @@ public class HiveRelOptUtil extends RelOptUtil {
             final int pos = posList.get(index);
             return fieldNames.get(pos);
           }
-        }, true, factory);
+        }, true, relBuilder);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java
deleted file mode 100644
index 15707c1..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java
+++ /dev/null
@@ -1,821 +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.hadoop.hive.ql.optimizer.calcite;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexShuttle;
-import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.ControlFlowException;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-
-
-public class HiveRexUtil {
-
-  protected static final Logger LOG = LoggerFactory.getLogger(HiveRexUtil.class);
-
-
-  /** Converts an expression to conjunctive normal form (CNF).
-   *
-   * <p>The following expression is in CNF:
-   *
-   * <blockquote>(a OR b) AND (c OR d)</blockquote>
-   *
-   * <p>The following expression is not in CNF:
-   *
-   * <blockquote>(a AND b) OR c</blockquote>
-   *
-   * but can be converted to CNF:
-   *
-   * <blockquote>(a OR c) AND (b OR c)</blockquote>
-   *
-   * <p>The following expression is not in CNF:
-   *
-   * <blockquote>NOT (a OR NOT b)</blockquote>
-   *
-   * but can be converted to CNF by applying de Morgan's theorem:
-   *
-   * <blockquote>NOT a AND b</blockquote>
-   *
-   * <p>Expressions not involving AND, OR or NOT at the top level are in CNF.
-   */
-  public static RexNode toCnf(RexBuilder rexBuilder, RexNode rex) {
-    return new CnfHelper(rexBuilder).toCnf(rex);
-  }
-
-  public static RexNode toCnf(RexBuilder rexBuilder, int maxCNFNodeCount, RexNode rex) {
-    return new CnfHelper(rexBuilder, maxCNFNodeCount).toCnf(rex);
-  }
-
-  /** Helps {@link org.apache.calcite.rex.RexUtil#toCnf}. */
-  private static class CnfHelper {
-    final RexBuilder rexBuilder;
-    int currentCount;
-    final int maxNodeCount;
-
-    private CnfHelper(RexBuilder rexBuilder) {
-      this(rexBuilder, Integer.MAX_VALUE);
-    }
-
-    private CnfHelper(RexBuilder rexBuilder, int maxNodeCount) {
-      this.rexBuilder = rexBuilder;
-      this.maxNodeCount = maxNodeCount == -1 ? Integer.MAX_VALUE : maxNodeCount;
-    }
-
-    public RexNode toCnf(RexNode rex) {
-      try {
-        this.currentCount = 0;
-        return toCnf2(rex);
-      } catch (OverflowError e) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Transformation to CNF not carried out as number of resulting nodes "
-                  + "in expression is greater than the max number of nodes allowed");
-        }
-        Util.swallow(e, null);
-        return rex;
-      }
-    }
-
-    private RexNode toCnf2(RexNode rex) {
-      final List<RexNode> operands;
-      switch (rex.getKind()) {
-      case AND:
-        incrementAndCheck();
-        operands = RexUtil.flattenAnd(((RexCall) rex).getOperands());
-        final List<RexNode> cnfOperands = Lists.newArrayList();
-        for (RexNode node : operands) {
-          RexNode cnf = toCnf2(node);
-          switch (cnf.getKind()) {
-          case AND:
-            incrementAndCheck();
-            cnfOperands.addAll(((RexCall) cnf).getOperands());
-            break;
-          default:
-            incrementAndCheck();
-            cnfOperands.add(cnf);
-          }
-        }
-        return and(cnfOperands);
-      case OR:
-        incrementAndCheck();
-        operands = RexUtil.flattenOr(((RexCall) rex).getOperands());
-        final RexNode head = operands.get(0);
-        final RexNode headCnf = toCnf2(head);
-        final List<RexNode> headCnfs = RelOptUtil.conjunctions(headCnf);
-        final RexNode tail = or(Util.skip(operands));
-        final RexNode tailCnf = toCnf2(tail);
-        final List<RexNode> tailCnfs = RelOptUtil.conjunctions(tailCnf);
-        final List<RexNode> list = Lists.newArrayList();
-        for (RexNode h : headCnfs) {
-          for (RexNode t : tailCnfs) {
-            list.add(or(ImmutableList.of(h, t)));
-          }
-        }
-        return and(list);
-      case NOT:
-        final RexNode arg = ((RexCall) rex).getOperands().get(0);
-        switch (arg.getKind()) {
-        case NOT:
-          return toCnf2(((RexCall) arg).getOperands().get(0));
-        case OR:
-          operands = ((RexCall) arg).getOperands();
-          List<RexNode> transformedDisj = new ArrayList<>();
-          for (RexNode input : RexUtil.flattenOr(operands)) {
-            transformedDisj.add(rexBuilder.makeCall(input.getType(), SqlStdOperatorTable.NOT,
-                    ImmutableList.of(input)));
-          }
-          return toCnf2(and(transformedDisj));
-        case AND:
-          operands = ((RexCall) arg).getOperands();
-          List<RexNode> transformedConj = new ArrayList<>();
-          for (RexNode input : RexUtil.flattenAnd(operands)) {
-            transformedConj.add(rexBuilder.makeCall(input.getType(), SqlStdOperatorTable.NOT,
-                    ImmutableList.of(input)));
-          }
-          return toCnf2(or(transformedConj));
-        default:
-          incrementAndCheck();
-          return rex;
-        }
-      default:
-        incrementAndCheck();
-        return rex;
-      }
-    }
-
-    private RexNode and(Iterable<? extends RexNode> nodes) {
-      return RexUtil.composeConjunction(rexBuilder, nodes, false);
-    }
-
-    private RexNode or(Iterable<? extends RexNode> nodes) {
-      return RexUtil.composeDisjunction(rexBuilder, nodes, false);
-    }
-
-    private void incrementAndCheck() {
-      this.currentCount++;
-      if (this.currentCount > this.maxNodeCount) {
-        throw OverflowError.INSTANCE;
-      }
-    }
-
-    @SuppressWarnings("serial")
-    private static class OverflowError extends ControlFlowException {
-
-      public static final OverflowError INSTANCE = new OverflowError();
-
-      private OverflowError() {}
-    }
-  }
-
-
-  /**
-   * Simplifies a boolean expression.
-   *
-   * <p>In particular:</p>
-   * <ul>
-   * <li>{@code simplify(x = 1 AND y = 2 AND NOT x = 1)}
-   * returns {@code y = 2}</li>
-   * <li>{@code simplify(x = 1 AND FALSE)}
-   * returns {@code FALSE}</li>
-   * </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, unknownAsFalse);
-    case OR:
-      return simplifyOr(rexBuilder, (RexCall) e);
-    case NOT:
-      return simplifyNot(rexBuilder, (RexCall) e);
-    case CASE:
-      return simplifyCase(rexBuilder, (RexCall) e, unknownAsFalse);
-    case IS_NULL:
-      return ((RexCall) e).getOperands().get(0).getType().isNullable()
-          ? e : rexBuilder.makeLiteral(false);
-    case IS_NOT_NULL:
-      return ((RexCall) e).getOperands().get(0).getType().isNullable()
-          ? e : rexBuilder.makeLiteral(true);
-    default:
-      return e;
-    }
-  }
-
-  private static RexNode simplifyNot(RexBuilder rexBuilder, RexCall call) {
-    final RexNode a = call.getOperands().get(0);
-    switch (a.getKind()) {
-    case NOT:
-      // NOT NOT x ==> x
-      return simplify(rexBuilder, ((RexCall) a).getOperands().get(0));
-    }
-    final SqlKind negateKind = a.getKind().negate();
-    if (a.getKind() != negateKind) {
-      return simplify(rexBuilder,
-          rexBuilder.makeCall(op(negateKind),
-              ImmutableList.of(((RexCall) a).getOperands().get(0))));
-    }
-    final SqlKind negateKind2 = negate(a.getKind());
-    if (a.getKind() != negateKind2) {
-      return simplify(rexBuilder,
-          rexBuilder.makeCall(op(negateKind2), ((RexCall) a).getOperands()));
-    }
-    if (a.getKind() == SqlKind.AND) {
-      // NOT distributivity for AND
-      final List<RexNode> newOperands = new ArrayList<>();
-      for (RexNode operand : ((RexCall) a).getOperands()) {
-        newOperands.add(simplify(rexBuilder,
-            rexBuilder.makeCall(SqlStdOperatorTable.NOT, operand)));
-      }
-      return simplify(rexBuilder,
-          rexBuilder.makeCall(SqlStdOperatorTable.OR, newOperands));
-    }
-    if (a.getKind() == SqlKind.OR) {
-      // NOT distributivity for OR
-      final List<RexNode> newOperands = new ArrayList<>();
-      for (RexNode operand : ((RexCall) a).getOperands()) {
-        newOperands.add(simplify(rexBuilder,
-            rexBuilder.makeCall(SqlStdOperatorTable.NOT, operand)));
-      }
-      return simplify(rexBuilder,
-          rexBuilder.makeCall(SqlStdOperatorTable.AND, newOperands));
-    }
-    return call;
-  }
-
-  private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call,
-          boolean unknownAsFalse) {
-    final List<RexNode> operands = call.getOperands();
-    final List<RexNode> newOperands = new ArrayList<>();
-    final Set<String> values = new HashSet<>();
-    for (int i = 0; i < operands.size(); i++) {
-      RexNode operand = operands.get(i);
-      if (RexUtil.isCasePredicate(call, i)) {
-        if (operand.isAlwaysTrue()) {
-          // Predicate is always TRUE. Make value the ELSE and quit.
-          newOperands.add(operands.get(i + 1));
-          if (unknownAsFalse && RexUtil.isNull(operands.get(i + 1))) {
-            values.add(rexBuilder.makeLiteral(false).toString());
-          } else {
-            values.add(operands.get(i + 1).toString());
-          }
-          break;
-        } else if (operand.isAlwaysFalse() || RexUtil.isNull(operand)) {
-          // Predicate is always FALSE or NULL. Skip predicate and value.
-          ++i;
-          continue;
-        }
-      } else {
-        if (unknownAsFalse && RexUtil.isNull(operand)) {
-          values.add(rexBuilder.makeLiteral(false).toString());
-        } else {
-          values.add(operand.toString());
-        }
-      }
-      newOperands.add(operand);
-    }
-    assert newOperands.size() % 2 == 1;
-    if (newOperands.size() == 1 || values.size() == 1) {
-      return rexBuilder.makeCast(call.getType(), newOperands.get(newOperands.size() - 1));
-    }
-  trueFalse:
-    if (call.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) {
-      // Optimize CASE where every branch returns constant true or constant
-      // false.
-      final List<Pair<RexNode, RexNode>> pairs =
-          casePairs(rexBuilder, newOperands);
-      // 1) Possible simplification if unknown is treated as false:
-      //   CASE
-      //   WHEN p1 THEN TRUE
-      //   WHEN p2 THEN TRUE
-      //   ELSE FALSE
-      //   END
-      // can be rewritten to: (p1 or p2)
-      if (unknownAsFalse) {
-        final List<RexNode> terms = new ArrayList<>();
-        int pos = 0;
-        for (; pos < pairs.size(); pos++) {
-          // True block
-          Pair<RexNode, RexNode> pair = pairs.get(pos);
-          if (!pair.getValue().isAlwaysTrue()) {
-            break;
-          }
-          terms.add(pair.getKey());
-        }
-        for (; pos < pairs.size(); pos++) {
-          // False block
-          Pair<RexNode, RexNode> pair = pairs.get(pos);
-          if (!pair.getValue().isAlwaysFalse() && !RexUtil.isNull(pair.getValue())) {
-            break;
-          }
-        }
-        if (pos == pairs.size()) {
-          return RexUtil.composeDisjunction(rexBuilder, terms, false);
-        }
-      }
-      // 2) Another simplification
-      //   CASE
-      //   WHEN p1 THEN TRUE
-      //   WHEN p2 THEN FALSE
-      //   WHEN p3 THEN TRUE
-      //   ELSE FALSE
-      //   END
-      // if p1...pn cannot be nullable
-      for (Ord<Pair<RexNode, RexNode>> pair : Ord.zip(pairs)) {
-        if (pair.e.getKey().getType().isNullable()) {
-          break trueFalse;
-        }
-        if (!pair.e.getValue().isAlwaysTrue()
-            && !pair.e.getValue().isAlwaysFalse()
-            && (!unknownAsFalse || !RexUtil.isNull(pair.e.getValue()))) {
-          break trueFalse;
-        }
-      }
-      final List<RexNode> terms = new ArrayList<>();
-      final List<RexNode> notTerms = new ArrayList<>();
-      for (Ord<Pair<RexNode, RexNode>> pair : Ord.zip(pairs)) {
-        if (pair.e.getValue().isAlwaysTrue()) {
-          terms.add(RexUtil.andNot(rexBuilder, pair.e.getKey(), notTerms));
-        } else {
-          notTerms.add(pair.e.getKey());
-        }
-      }
-      return RexUtil.composeDisjunction(rexBuilder, terms, false);
-    }
-    if (newOperands.equals(operands)) {
-      return call;
-    }
-    return call.clone(call.getType(), newOperands);
-  }
-
-  /** Given "CASE WHEN p1 THEN v1 ... ELSE e END"
-   * returns [(p1, v1), ..., (true, e)]. */
-  private static List<Pair<RexNode, RexNode>> casePairs(RexBuilder rexBuilder,
-      List<RexNode> operands) {
-    final ImmutableList.Builder<Pair<RexNode, RexNode>> builder =
-        ImmutableList.builder();
-    for (int i = 0; i < operands.size() - 1; i += 2) {
-      builder.add(Pair.of(operands.get(i), operands.get(i + 1)));
-    }
-    builder.add(
-        Pair.of((RexNode) rexBuilder.makeLiteral(true), Util.last(operands)));
-    return builder.build();
-  }
-
-  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);
-  }
-
-  public static RexNode simplifyAnd2(RexBuilder rexBuilder,
-      List<RexNode> terms, List<RexNode> notTerms) {
-    for (RexNode term : terms) {
-      if (term.isAlwaysFalse()) {
-        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));
-    }
-    // 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
-    for (RexNode notDisjunction : notTerms) {
-      final List<RexNode> terms2 = RelOptUtil.conjunctions(notDisjunction);
-      if (terms.containsAll(terms2)) {
-        return rexBuilder.makeLiteral(false);
-      }
-    }
-    // Add the NOT disjunctions back in.
-    for (RexNode notDisjunction : notTerms) {
-      terms.add(
-          simplify(rexBuilder,
-              rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction)));
-    }
-    return RexUtil.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) {
-    for (RexNode term : terms) {
-      if (term.isAlwaysFalse()) {
-        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 Multimap<String,Pair<String,RexNode>> equalityTerms = ArrayListMultimap.create();
-    final Map<String,String> equalityConstantTerms = new HashMap<>();
-    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++) {
-      RexNode term = terms.get(i);
-      if (!HiveCalciteUtil.isDeterministic(term)) {
-        continue;
-      }
-      // Simplify BOOLEAN expressions if possible
-      while (term.getKind() == SqlKind.EQUALS) {
-        RexCall call = (RexCall) term;
-        if (call.getOperands().get(0).isAlwaysTrue()) {
-          term = call.getOperands().get(1);
-          terms.remove(i);
-          terms.add(i, term);
-          continue;
-        } else if (call.getOperands().get(1).isAlwaysTrue()) {
-          term = call.getOperands().get(0);
-          terms.remove(i);
-          terms.add(i, term);
-          continue;
-        }
-        break;
-      }
-      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());
-        RexCall leftCast = null;
-        // if it is a cast, we include the inner reference
-        if (left.getKind() == SqlKind.CAST) {
-          leftCast = (RexCall) left;
-          comparedOperands.add(leftCast.getOperands().get(0).toString());
-        }
-        RexNode right = call.getOperands().get(1);
-        comparedOperands.add(right.toString());
-        RexCall rightCast = null;
-        // if it is a cast, we include the inner reference
-        if (right.getKind() == SqlKind.CAST) {
-          rightCast = (RexCall) right;
-          comparedOperands.add(rightCast.getOperands().get(0).toString());
-        }
-        // Check for equality on different constants. If the same ref or CAST(ref)
-        // is equal to different constants, this condition cannot be satisfied,
-        // and hence it can be evaluated to FALSE
-        if (term.getKind() == SqlKind.EQUALS) {
-          boolean leftRef = left instanceof RexInputRef ||
-                  (leftCast != null && leftCast.getOperands().get(0) instanceof RexInputRef);
-          boolean rightRef = right instanceof RexInputRef ||
-                  (rightCast != null && rightCast.getOperands().get(0) instanceof RexInputRef);
-          if (right instanceof RexLiteral && leftRef) {
-            final String literal = right.toString();
-            final String prevLiteral = equalityConstantTerms.put(left.toString(), literal);
-            if (prevLiteral != null && !literal.equals(prevLiteral)) {
-              return rexBuilder.makeLiteral(false);
-            }
-          } else if (left instanceof RexLiteral && rightRef) {
-            final String literal = left.toString();
-            final String prevLiteral = equalityConstantTerms.put(right.toString(), literal);
-            if (prevLiteral != null && !literal.equals(prevLiteral)) {
-              return rexBuilder.makeLiteral(false);
-            }
-          } else if (leftRef && rightRef) {
-            equalityTerms.put(left.toString(), Pair.of(right.toString(), term));
-          }
-        }
-        // 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);
-    }
-    // Check for equality of two refs wrt equality with constants
-    // Example #1. x=5 AND y=5 AND x=y : x=5 AND y=5
-    // Example #2. x=5 AND y=6 AND x=y - not satisfiable
-    for (String ref1 : equalityTerms.keySet()) {
-      final String literal1 = equalityConstantTerms.get(ref1);
-      if (literal1 == null) {
-        continue;
-      }
-      Collection<Pair<String, RexNode>> references = equalityTerms.get(ref1);
-      for (Pair<String,RexNode> ref2 : references) {
-        final String literal2 = equalityConstantTerms.get(ref2.left);
-        if (literal2 == null) {
-          continue;
-        }
-        if (!literal1.equals(literal2)) {
-          // If an expression is equal to two different constants,
-          // it is not satisfiable
-          return rexBuilder.makeLiteral(false);
-        }
-        // Otherwise we can remove the term, as we already know that
-        // the expression is equal to two constants
-        terms.remove(ref2.right);
-      }
-    }
-    // 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 = new HashSet<String>(
-            Lists.transform(terms, HiveCalciteUtil.REX_STR_FN));
-    for (RexNode notDisjunction : notTerms) {
-      if (!HiveCalciteUtil.isDeterministic(notDisjunction)) {
-        continue;
-      }
-      final List<String> terms2Set = Lists.transform(
-              RelOptUtil.conjunctions(notDisjunction), HiveCalciteUtil.REX_STR_FN);
-      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 RexUtil.composeConjunction(rexBuilder, terms, false);
-  }
-
-  /** Simplifies OR(x, x) into x, and similar. */
-  public static RexNode simplifyOr(RexBuilder rexBuilder, RexCall call) {
-    assert call.getKind() == SqlKind.OR;
-    final List<RexNode> terms = RelOptUtil.disjunctions(call);
-    for (int i = 0; i < terms.size(); i++) {
-      final RexNode term = terms.get(i);
-      switch (term.getKind()) {
-      case LITERAL:
-        if (!RexLiteral.isNullLiteral(term)) {
-          if (RexLiteral.booleanValue(term)) {
-            return term; // true
-          } else {
-            terms.remove(i);
-            --i;
-          }
-        }
-      }
-    }
-    return RexUtil.composeDisjunction(rexBuilder, terms, false);
-  }
-
-  private static RexCall 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:
-        return (RexCall) rexBuilder.makeCall(op(negate(call.getKind())), call.getOperands());
-    }
-    return null;
-  }
-
-  private static SqlKind negate(SqlKind kind) {
-    switch (kind) {
-      case EQUALS:
-        return SqlKind.NOT_EQUALS;
-      case NOT_EQUALS:
-        return SqlKind.EQUALS;
-      case LESS_THAN:
-        return SqlKind.GREATER_THAN_OR_EQUAL;
-      case GREATER_THAN:
-        return SqlKind.LESS_THAN_OR_EQUAL;
-      case LESS_THAN_OR_EQUAL:
-        return SqlKind.GREATER_THAN;
-      case GREATER_THAN_OR_EQUAL:
-        return SqlKind.LESS_THAN;
-    }
-    return kind;
-  }
-
-  private static RexCall invert(RexBuilder rexBuilder, RexCall call) {
-    switch (call.getKind()) {
-      case EQUALS:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                Lists.reverse(call.getOperands()));
-      case NOT_EQUALS:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.NOT_EQUALS,
-                Lists.reverse(call.getOperands()));
-      case LESS_THAN:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN,
-                Lists.reverse(call.getOperands()));
-      case GREATER_THAN:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN,
-                Lists.reverse(call.getOperands()));
-      case LESS_THAN_OR_EQUAL:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-                Lists.reverse(call.getOperands()));
-      case GREATER_THAN_OR_EQUAL:
-        return (RexCall) rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-                Lists.reverse(call.getOperands()));
-    }
-    return null;
-  }
-
-  private static SqlOperator op(SqlKind kind) {
-    switch (kind) {
-    case IS_FALSE:
-      return SqlStdOperatorTable.IS_FALSE;
-    case IS_TRUE:
-      return SqlStdOperatorTable.IS_TRUE;
-    case IS_UNKNOWN:
-      return SqlStdOperatorTable.IS_UNKNOWN;
-    case IS_NULL:
-      return SqlStdOperatorTable.IS_NULL;
-    case IS_NOT_FALSE:
-      return SqlStdOperatorTable.IS_NOT_FALSE;
-    case IS_NOT_TRUE:
-      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);
-    }
-  }
-
-  public static SqlKind invert(SqlKind kind) {
-    switch (kind) {
-      case EQUALS:
-        return SqlKind.EQUALS;
-      case NOT_EQUALS:
-        return SqlKind.NOT_EQUALS;
-      case LESS_THAN:
-        return SqlKind.GREATER_THAN;
-      case GREATER_THAN:
-        return SqlKind.LESS_THAN;
-      case LESS_THAN_OR_EQUAL:
-        return SqlKind.GREATER_THAN_OR_EQUAL;
-      case GREATER_THAN_OR_EQUAL:
-        return SqlKind.LESS_THAN_OR_EQUAL;
-    }
-    return null;
-  }
-
-  public static class ExprSimplifier extends RexShuttle {
-    private final RexBuilder rexBuilder;
-    private final boolean unknownAsFalse;
-    private final Map<RexNode,Boolean> unknownAsFalseMap;
-
-    public ExprSimplifier(RexBuilder rexBuilder, boolean unknownAsFalse) {
-      this.rexBuilder = rexBuilder;
-      this.unknownAsFalse = unknownAsFalse;
-      this.unknownAsFalseMap = new HashMap<>();
-    }
-
-    @Override
-    public RexNode visitCall(RexCall call) {
-      Boolean unknownAsFalseCall = unknownAsFalse;
-      if (unknownAsFalseCall) {
-        switch (call.getKind()) {
-        case AND:
-        case CASE:
-          unknownAsFalseCall = this.unknownAsFalseMap.get(call);
-          if (unknownAsFalseCall == null) {
-            // Top operator
-            unknownAsFalseCall = true;
-          }
-          break;
-        default:
-          unknownAsFalseCall = false;
-        }
-        for (RexNode operand : call.operands) {
-          this.unknownAsFalseMap.put(operand, unknownAsFalseCall);
-        }
-      }
-      RexNode node = super.visitCall(call);
-      RexNode simplifiedNode = HiveRexUtil.simplify(rexBuilder, node, unknownAsFalseCall);
-      if (node == simplifiedNode) {
-        return node;
-      }
-      if (simplifiedNode.getType().equals(call.getType())) {
-        return simplifiedNode;
-      }
-      return rexBuilder.makeCast(call.getType(), simplifiedNode, true);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveTypeSystemImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveTypeSystemImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveTypeSystemImpl.java
index 10fdcc6..279d101 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveTypeSystemImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveTypeSystemImpl.java
@@ -36,8 +36,19 @@ public class HiveTypeSystemImpl extends RelDataTypeSystemImpl {
     switch (typeName) {
     case DECIMAL:
       return getMaxNumericScale();
-    case INTERVAL_DAY_TIME:
+    case INTERVAL_YEAR:
+    case INTERVAL_MONTH:
     case INTERVAL_YEAR_MONTH:
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_SECOND:
       return SqlTypeName.MAX_INTERVAL_FRACTIONAL_SECOND_PRECISION;
     default:
       return -1;
@@ -58,8 +69,19 @@ public class HiveTypeSystemImpl extends RelDataTypeSystemImpl {
       return getMaxPrecision(typeName);
     case DECIMAL:
       return DEFAULT_DECIMAL_PRECISION;
-    case INTERVAL_DAY_TIME:
+    case INTERVAL_YEAR:
+    case INTERVAL_MONTH:
     case INTERVAL_YEAR_MONTH:
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_SECOND:
       return SqlTypeName.DEFAULT_INTERVAL_START_PRECISION;
     default:
       return -1;
@@ -81,8 +103,19 @@ public class HiveTypeSystemImpl extends RelDataTypeSystemImpl {
     case TIME:
     case TIMESTAMP:
       return MAX_TIMESTAMP_PRECISION;
-    case INTERVAL_DAY_TIME:
+    case INTERVAL_YEAR:
+    case INTERVAL_MONTH:
     case INTERVAL_YEAR_MONTH:
+    case INTERVAL_DAY:
+    case INTERVAL_DAY_HOUR:
+    case INTERVAL_DAY_MINUTE:
+    case INTERVAL_DAY_SECOND:
+    case INTERVAL_HOUR:
+    case INTERVAL_HOUR_MINUTE:
+    case INTERVAL_HOUR_SECOND:
+    case INTERVAL_MINUTE:
+    case INTERVAL_MINUTE_SECOND:
+    case INTERVAL_SECOND:
       return SqlTypeName.MAX_INTERVAL_START_PRECISION;
     default:
       return -1;

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveDefaultCostModel.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveDefaultCostModel.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveDefaultCostModel.java
index badb8ca..40f2cef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveDefaultCostModel.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveDefaultCostModel.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.cost;
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
@@ -92,12 +93,12 @@ public class HiveDefaultCostModel extends HiveCostModel {
 
     @Override
     public ImmutableList<RelCollation> getCollation(HiveJoin join) {
-      return null;
+      return ImmutableList.of();
     }
 
     @Override
     public RelDistribution getDistribution(HiveJoin join) {
-      return null;
+      return RelDistributions.SINGLETON;
     }
 
     @Override
@@ -117,7 +118,7 @@ public class HiveDefaultCostModel extends HiveCostModel {
 
     @Override
     public Integer getSplitCount(HiveJoin join) {
-      return null;
+      return 1;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveRelMdCost.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveRelMdCost.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveRelMdCost.java
index ed45ab3..cbea307 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveRelMdCost.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/cost/HiveRelMdCost.java
@@ -19,7 +19,10 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.cost;
 
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.BuiltInMetadata;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.calcite.rel.metadata.MetadataHandler;
 import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMdPercentageOriginalRows;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -34,7 +37,7 @@ import com.google.common.collect.ImmutableList;
 /**
  * HiveRelMdCost supplies the implementation of cost model.
  */
-public class HiveRelMdCost {
+public class HiveRelMdCost implements MetadataHandler<BuiltInMetadata.NonCumulativeCost> {
 
   private final HiveCostModel hiveCostModel;
 
@@ -50,6 +53,11 @@ public class HiveRelMdCost {
                    RelMdPercentageOriginalRows.SOURCE));
   }
 
+  @Override
+  public MetadataDef<BuiltInMetadata.NonCumulativeCost> getDef() {
+    return BuiltInMetadata.NonCumulativeCost.DEF;
+  }
+
   public RelOptCost getNonCumulativeCost(HiveAggregate aggregate, RelMetadataQuery mq) {
     return hiveCostModel.getAggregateCost(aggregate);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
deleted file mode 100644
index 82ab4d7..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
+++ /dev/null
@@ -1,466 +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.hadoop.hive.ql.optimizer.calcite.druid;
-
-import java.sql.Timestamp;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.TreeSet;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.commons.lang.StringUtils;
-import org.joda.time.Interval;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Function;
-import com.google.common.collect.BoundType;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Range;
-import com.google.common.collect.Sets;
-
-/** 
- * Utilities for generating intervals from RexNode.
- * 
- * Based on Navis logic implemented on Hive data structures.
- * See <a href="https://github.com/druid-io/druid/pull/2880">Druid PR-2880</a>
- * 
- */
-@SuppressWarnings({"rawtypes","unchecked"})
-public class DruidIntervalUtils {
-
-  protected static final Logger LOG = LoggerFactory.getLogger(DruidIntervalUtils.class);
-
-
-  /**
-   * Given a list of predicates, it generates the equivalent Interval
-   * (if possible). It assumes that all the predicates in the input
-   * reference a single column : the timestamp column.
-   * 
-   * @param conjs list of conditions to use for the transformation
-   * @return interval representing the conditions in the input list
-   */
-  public static List<Interval> createInterval(RelDataType type, List<RexNode> conjs) {
-    List<Range> ranges = new ArrayList<>();
-    for (RexNode child : conjs) {
-      List<Range> extractedRanges = extractRanges(type, child, false);
-      if (extractedRanges == null || extractedRanges.isEmpty()) {
-        // We could not extract, we bail out
-        return null;
-      }
-      if (ranges.isEmpty()) {
-        ranges.addAll(extractedRanges);
-        continue;
-      }
-      List<Range> overlapped = Lists.newArrayList();
-      for (Range current : ranges) {
-        for (Range interval : extractedRanges) {
-          if (current.isConnected(interval)) {
-            overlapped.add(current.intersection(interval));
-          }
-        }
-      }
-      ranges = overlapped;
-    }
-    List<Range> compactRanges = condenseRanges(ranges);
-    LOG.debug("Inferred ranges on interval : " + compactRanges);
-    return toInterval(compactRanges);
-  }
-
-  protected static List<Interval> toInterval(List<Range> ranges) {
-    List<Interval> intervals = Lists.transform(ranges, new Function<Range, Interval>() {
-      @Override
-      public Interval apply(Range range) {
-        if (!range.hasLowerBound() && !range.hasUpperBound()) {
-          return DruidTable.DEFAULT_INTERVAL;
-        }
-        long start = range.hasLowerBound() ? toLong(range.lowerEndpoint()) :
-          DruidTable.DEFAULT_INTERVAL.getStartMillis();
-        long end = range.hasUpperBound() ? toLong(range.upperEndpoint()) :
-          DruidTable.DEFAULT_INTERVAL.getEndMillis();
-        if (range.hasLowerBound() && range.lowerBoundType() == BoundType.OPEN) {
-          start++;
-        }
-        if (range.hasUpperBound() && range.upperBoundType() == BoundType.CLOSED) {
-          end++;
-        }
-        return new Interval(start, end);
-      }
-    });
-    LOG.info("Converted time ranges " + ranges + " to interval " + intervals);
-    return intervals;
-  }
-
-  protected static List<Range> extractRanges(RelDataType type, RexNode node,
-          boolean withNot) {
-    switch (node.getKind()) {
-      case EQUALS:
-      case LESS_THAN:
-      case LESS_THAN_OR_EQUAL:
-      case GREATER_THAN:
-      case GREATER_THAN_OR_EQUAL:
-      case BETWEEN:
-      case IN:
-        return leafToRanges(type, (RexCall) node, withNot);
-
-      case NOT:
-        return extractRanges(type, ((RexCall) node).getOperands().get(0), !withNot);
-
-      case OR:
-        RexCall call = (RexCall) node;
-        List<Range> intervals = Lists.newArrayList();
-        for (RexNode child : call.getOperands()) {
-          List<Range> extracted = extractRanges(type, child, withNot);
-          if (extracted != null) {
-            intervals.addAll(extracted);
-          }
-        }
-        return intervals;
-
-      default:
-        return null;
-    }
-  }
-
-  protected static List<Range> leafToRanges(RelDataType type, RexCall call,
-          boolean withNot) {
-    switch (call.getKind()) {
-      case EQUALS:
-      case LESS_THAN:
-      case LESS_THAN_OR_EQUAL:
-      case GREATER_THAN:
-      case GREATER_THAN_OR_EQUAL:
-      {
-        RexLiteral literal = null;
-        if (call.getOperands().get(0) instanceof RexInputRef &&
-                call.getOperands().get(1) instanceof RexLiteral) {
-          literal = extractLiteral(call.getOperands().get(1));
-        } else if (call.getOperands().get(0) instanceof RexInputRef &&
-                call.getOperands().get(1).getKind() == SqlKind.CAST) {
-          literal = extractLiteral(call.getOperands().get(1));
-        } else if (call.getOperands().get(1) instanceof RexInputRef &&
-                call.getOperands().get(0) instanceof RexLiteral) {
-          literal = extractLiteral(call.getOperands().get(0));
-        } else if (call.getOperands().get(1) instanceof RexInputRef &&
-                call.getOperands().get(0).getKind() == SqlKind.CAST) {
-          literal = extractLiteral(call.getOperands().get(0));
-        }
-        if (literal == null) {
-          return null;
-        }
-        Comparable value = literalToType(literal, type);
-        if (value == null) {
-          return null;
-        }
-        if (call.getKind() == SqlKind.LESS_THAN) {
-          return Arrays.<Range> asList(withNot ? Range.atLeast(value) : Range.lessThan(value));
-        } else if (call.getKind() == SqlKind.LESS_THAN_OR_EQUAL) {
-          return Arrays.<Range> asList(withNot ? Range.greaterThan(value) : Range.atMost(value));
-        } else if (call.getKind() == SqlKind.GREATER_THAN) {
-          return Arrays.<Range> asList(withNot ? Range.atMost(value) : Range.greaterThan(value));
-        } else if (call.getKind() == SqlKind.GREATER_THAN_OR_EQUAL) {
-          return Arrays.<Range> asList(withNot ? Range.lessThan(value) : Range.atLeast(value));
-        } else { //EQUALS
-          if (!withNot) {
-            return Arrays.<Range> asList(Range.closed(value, value));
-          }
-          return Arrays.<Range> asList(Range.lessThan(value), Range.greaterThan(value));
-        }
-      }
-      case BETWEEN:
-      {
-        RexLiteral literal1 = extractLiteral(call.getOperands().get(2));
-        if (literal1 == null) {
-          return null;
-        }
-        RexLiteral literal2 = extractLiteral(call.getOperands().get(3));
-        if (literal2 == null) {
-          return null;
-        }
-        Comparable value1 = literalToType(literal1, type);
-        Comparable value2 = literalToType(literal2, type);
-        if (value1 == null || value2 == null) {
-          return null;
-        }
-        boolean inverted = value1.compareTo(value2) > 0;
-        if (!withNot) {
-          return Arrays.<Range> asList(
-                  inverted ? Range.closed(value2, value1) : Range.closed(value1, value2));
-        }
-        return Arrays.<Range> asList(Range.lessThan(inverted ? value2 : value1),
-                Range.greaterThan(inverted ? value1 : value2));
-      }
-      case IN:
-      {
-        List<Range> ranges = Lists.newArrayList();
-        for (int i = 1; i < call.getOperands().size(); i++) {
-          RexLiteral literal = extractLiteral(call.getOperands().get(i));
-          if (literal == null) {
-            return null;
-          }
-          Comparable element = literalToType(literal, type);
-          if (element == null) {
-            return null;
-          }
-          if (withNot) {
-            ranges.addAll(
-                    Arrays.<Range> asList(Range.lessThan(element), Range.greaterThan(element)));
-          } else {
-            ranges.add(Range.closed(element, element));
-          }
-        }
-        return ranges;
-      }
-      default:
-        return null;
-    }
-  }
-
-  @SuppressWarnings("incomplete-switch")
-  protected static Comparable literalToType(RexLiteral literal, RelDataType type) {
-    // Extract
-    Object value = null;
-    switch (literal.getType().getSqlTypeName()) {
-      case DATE:
-      case TIME:
-      case TIMESTAMP:
-      case INTERVAL_YEAR_MONTH:
-      case INTERVAL_DAY_TIME:
-        value = literal.getValue();
-        break;
-      case TINYINT:
-      case SMALLINT:
-      case INTEGER:
-      case BIGINT:
-      case DOUBLE:
-      case DECIMAL:
-      case FLOAT:
-      case REAL:
-      case VARCHAR:
-      case CHAR:
-      case BOOLEAN:
-        value = literal.getValue3();
-    }
-    if (value == null) {
-      return null;
-    }
-
-    // Convert
-    switch (type.getSqlTypeName()) {
-      case BIGINT:
-        return toLong(value);
-      case INTEGER:
-        return toInt(value);
-      case FLOAT:
-        return toFloat(value);
-      case DOUBLE:
-        return toDouble(value);
-      case VARCHAR:
-      case CHAR:
-        return String.valueOf(value);
-      case TIMESTAMP:
-        return toTimestamp(value);
-    }
-    return null;
-  }
-
-  private static RexLiteral extractLiteral(RexNode node) {
-    RexNode target = node;
-    if (node.getKind() == SqlKind.CAST) {
-      target = ((RexCall)node).getOperands().get(0);
-    }
-    if (!(target instanceof RexLiteral)) {
-      return null;
-    }
-    return (RexLiteral) target;
-  }
-
-  private static Comparable toTimestamp(Object literal) {
-    if (literal instanceof Timestamp) {
-      return (Timestamp) literal;
-    }
-    if (literal instanceof Date) {
-      return new Timestamp(((Date) literal).getTime());
-    }
-    if (literal instanceof Number) {
-      return new Timestamp(((Number) literal).longValue());
-    }
-    if (literal instanceof String) {
-      String string = (String) literal;
-      if (StringUtils.isNumeric(string)) {
-        return new Timestamp(Long.valueOf(string));
-      }
-      try {
-        return Timestamp.valueOf(string);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
-    }
-    return null;
-  }
-
-  private static Long toLong(Object literal) {
-    if (literal instanceof Number) {
-      return ((Number) literal).longValue();
-    }
-    if (literal instanceof Date) {
-      return ((Date) literal).getTime();
-    }
-    if (literal instanceof Timestamp) {
-      return ((Timestamp) literal).getTime();
-    }
-    if (literal instanceof String) {
-      try {
-        return Long.valueOf((String) literal);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
-      try {
-        return DateFormat.getDateInstance().parse((String) literal).getTime();
-      } catch (ParseException e) {
-        // best effort. ignore
-      }
-    }
-    return null;
-  }
-
-
-  private static Integer toInt(Object literal) {
-    if (literal instanceof Number) {
-      return ((Number) literal).intValue();
-    }
-    if (literal instanceof String) {
-      try {
-        return Integer.valueOf((String) literal);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
-    }
-    return null;
-  }
-
-  private static Float toFloat(Object literal) {
-    if (literal instanceof Number) {
-      return ((Number) literal).floatValue();
-    }
-    if (literal instanceof String) {
-      try {
-        return Float.valueOf((String) literal);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
-    }
-    return null;
-  }
-
-  private static Double toDouble(Object literal) {
-    if (literal instanceof Number) {
-      return ((Number) literal).doubleValue();
-    }
-    if (literal instanceof String) {
-      try {
-        return Double.valueOf((String) literal);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
-    }
-    return null;
-  }
-
-  protected static List<Range> condenseRanges(List<Range> ranges) {
-    if (ranges.size() <= 1) {
-      return ranges;
-    }
-
-    Comparator<Range> startThenEnd = new Comparator<Range>() {
-      @Override
-      public int compare(Range lhs, Range rhs) {
-        int compare = 0;
-        if (lhs.hasLowerBound() && rhs.hasLowerBound()) {
-          compare = lhs.lowerEndpoint().compareTo(rhs.lowerEndpoint());
-        } else if (!lhs.hasLowerBound() && rhs.hasLowerBound()) {
-          compare = -1;
-        } else if (lhs.hasLowerBound() && !rhs.hasLowerBound()) {
-          compare = 1;
-        }
-        if (compare != 0) {
-          return compare;
-        }
-        if (lhs.hasUpperBound() && rhs.hasUpperBound()) {
-          compare = lhs.upperEndpoint().compareTo(rhs.upperEndpoint());
-        } else if (!lhs.hasUpperBound() && rhs.hasUpperBound()) {
-          compare = -1;
-        } else if (lhs.hasUpperBound() && !rhs.hasUpperBound()) {
-          compare = 1;
-        }
-        return compare;
-      }
-    };
-
-    TreeSet<Range> sortedIntervals = Sets.newTreeSet(startThenEnd);
-    sortedIntervals.addAll(ranges);
-
-    List<Range> retVal = Lists.newArrayList();
-
-    Iterator<Range> intervalsIter = sortedIntervals.iterator();
-    Range currInterval = intervalsIter.next();
-    while (intervalsIter.hasNext()) {
-      Range next = intervalsIter.next();
-      if (currInterval.encloses(next)) {
-        continue;
-      }
-      if (mergeable(currInterval, next)) {
-        currInterval = currInterval.span(next);
-      } else {
-        retVal.add(currInterval);
-        currInterval = next;
-      }
-    }
-    retVal.add(currInterval);
-
-    return retVal;
-  }
-
-  protected static boolean mergeable(Range range1, Range range2) {
-    Comparable x1 = range1.upperEndpoint();
-    Comparable x2 = range2.lowerEndpoint();
-    int compare = x1.compareTo(x2);
-    return compare > 0 || (compare == 0 && range1.upperBoundType() == BoundType.CLOSED
-            && range2.lowerBoundType() == BoundType.CLOSED);
-  }
-
-  public static long extractTotalTime(List<Interval> intervals) {
-    long totalTime = 0;
-    for (Interval interval : intervals) {
-      totalTime += (interval.getEndMillis() - interval.getStartMillis());
-    }
-    return totalTime;
-  }
-
-}


[2/6] hive git commit: HIVE-13316: Upgrade to Calcite 1.10 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/explain_logical.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_logical.q.out b/ql/src/test/results/clientpositive/explain_logical.q.out
index 4a25a38..79a3050 100644
--- a/ql/src/test/results/clientpositive/explain_logical.q.out
+++ b/ql/src/test/results/clientpositive/explain_logical.q.out
@@ -364,36 +364,37 @@ PREHOOK: type: QUERY
 POSTHOOK: query: EXPLAIN LOGICAL SELECT * FROM V4
 POSTHOOK: type: QUERY
 LOGICAL PLAN:
-$hdt$_0:src 
+$hdt$_0:srcpart 
   TableScan (TS_0)
-    alias: src
+    alias: srcpart
     properties:
       insideView TRUE
-    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+    Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
     Filter Operator (FIL_15)
       predicate: key is not null (type: boolean)
-      Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+      Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
       Select Operator (SEL_2)
-        expressions: key (type: string)
-        outputColumnNames: _col0
-        Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+        expressions: key (type: string), value (type: string)
+        outputColumnNames: _col0, _col1
+        Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
         Reduce Output Operator (RS_9)
           key expressions: _col0 (type: string)
           sort order: +
           Map-reduce partition columns: _col0 (type: string)
-          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
+          value expressions: _col1 (type: string)
           Join Operator (JOIN_12)
             condition map:
                  Inner Join 0 to 1
-                 Inner Join 1 to 2
+                 Inner Join 0 to 2
             keys:
               0 _col0 (type: string)
               1 _col0 (type: string)
               2 _col0 (type: string)
-            outputColumnNames: _col0, _col2, _col4
+            outputColumnNames: _col1, _col2, _col4
             Statistics: Num rows: 4400 Data size: 46745 Basic stats: COMPLETE Column stats: NONE
             Select Operator (SEL_13)
-              expressions: _col0 (type: string), _col2 (type: string), _col4 (type: string)
+              expressions: _col2 (type: string), _col1 (type: string), _col4 (type: string)
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 4400 Data size: 46745 Basic stats: COMPLETE Column stats: NONE
               File Output Operator (FS_14)
@@ -403,34 +404,33 @@ $hdt$_0:src
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                     serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-$hdt$_1:srcpart 
+$hdt$_1:src 
   TableScan (TS_3)
-    alias: srcpart
+    alias: src
     properties:
       insideView TRUE
-    Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
+    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
     Filter Operator (FIL_16)
       predicate: key is not null (type: boolean)
-      Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
+      Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
       Select Operator (SEL_5)
-        expressions: key (type: string), value (type: string)
-        outputColumnNames: _col0, _col1
-        Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
+        expressions: key (type: string)
+        outputColumnNames: _col0
+        Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
         Reduce Output Operator (RS_10)
           key expressions: _col0 (type: string)
           sort order: +
           Map-reduce partition columns: _col0 (type: string)
-          Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
-          value expressions: _col1 (type: string)
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
           Join Operator (JOIN_12)
             condition map:
                  Inner Join 0 to 1
-                 Inner Join 1 to 2
+                 Inner Join 0 to 2
             keys:
               0 _col0 (type: string)
               1 _col0 (type: string)
               2 _col0 (type: string)
-            outputColumnNames: _col0, _col2, _col4
+            outputColumnNames: _col1, _col2, _col4
             Statistics: Num rows: 4400 Data size: 46745 Basic stats: COMPLETE Column stats: NONE
 $hdt$_2:src3 
   TableScan (TS_6)
@@ -454,12 +454,12 @@ $hdt$_2:src3
           Join Operator (JOIN_12)
             condition map:
                  Inner Join 0 to 1
-                 Inner Join 1 to 2
+                 Inner Join 0 to 2
             keys:
               0 _col0 (type: string)
               1 _col0 (type: string)
               2 _col0 (type: string)
-            outputColumnNames: _col0, _col2, _col4
+            outputColumnNames: _col1, _col2, _col4
             Statistics: Num rows: 4400 Data size: 46745 Basic stats: COMPLETE Column stats: NONE
 
 PREHOOK: query: -- The table should show up in the explain logical even if none

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out b/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
index e70f912..6572f6c 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
@@ -1387,11 +1387,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: key
+              outputColumnNames: _col1
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(key)
-                keys: key (type: string)
+                aggregations: count(1)
+                keys: _col1 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -1482,7 +1482,7 @@ STAGE PLANS:
               name: default.t1
             name: default.t1
       Truncated Path -> Alias:
-        /t1 [t1]
+        /t1 [$hdt$_0:t1]
 
   Stage: Stage-7
     Conditional Operator
@@ -1708,7 +1708,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@outputtbl3
-POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.null, ]
 POSTHOOK: Lineage: outputtbl3.key1 SIMPLE []
 POSTHOOK: Lineage: outputtbl3.key2 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
 PREHOOK: query: SELECT * FROM outputTbl3
@@ -1757,11 +1757,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: key, val
+              outputColumnNames: _col0, _col2
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(val)
-                keys: key (type: string), val (type: string)
+                aggregations: count(1)
+                keys: _col0 (type: string), _col2 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1826,7 +1826,7 @@ STAGE PLANS:
               name: default.t1
             name: default.t1
       Truncated Path -> Alias:
-        /t1 [t1]
+        /t1 [$hdt$_0:t1]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -1912,7 +1912,7 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@outputtbl4
-POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.null, ]
 POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl4.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ]
@@ -4123,11 +4123,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: key, val
+              outputColumnNames: _col0, _col2
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(val)
-                keys: key (type: string), val (type: string)
+                aggregations: count(1)
+                keys: _col0 (type: string), _col2 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4218,7 +4218,7 @@ STAGE PLANS:
               name: default.t2
             name: default.t2
       Truncated Path -> Alias:
-        /t2 [t2]
+        /t2 [$hdt$_0:t2]
 
   Stage: Stage-7
     Conditional Operator
@@ -4444,7 +4444,7 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t2
 POSTHOOK: Output: default@outputtbl4
-POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.null, ]
 POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl4.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
@@ -4502,11 +4502,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: key, val
+              outputColumnNames: _col0, _col2
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(val)
-                keys: key (type: string), val (type: string)
+                aggregations: count(1)
+                keys: _col0 (type: string), _col2 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4597,7 +4597,7 @@ STAGE PLANS:
               name: default.t2
             name: default.t2
       Truncated Path -> Alias:
-        /t2 [t2]
+        /t2 [$hdt$_0:t2]
 
   Stage: Stage-7
     Conditional Operator
@@ -4823,7 +4823,7 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t2
 POSTHOOK: Output: default@outputtbl5
-POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.null, ]
 POSTHOOK: Lineage: outputtbl5.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl5.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl5.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out b/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
index fc52984..ce71354 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
@@ -1453,11 +1453,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: key
+              outputColumnNames: _col1
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(key)
-                keys: key (type: string)
+                aggregations: count(1)
+                keys: _col1 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -1548,7 +1548,7 @@ STAGE PLANS:
               name: default.t1
             name: default.t1
       Truncated Path -> Alias:
-        /t1 [t1]
+        /t1 [$hdt$_0:t1]
 
   Stage: Stage-7
     Conditional Operator
@@ -1774,7 +1774,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@outputtbl3
-POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.null, ]
 POSTHOOK: Lineage: outputtbl3.key1 SIMPLE []
 POSTHOOK: Lineage: outputtbl3.key2 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
 PREHOOK: query: SELECT * FROM outputTbl3
@@ -1824,11 +1824,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: key, val
+              outputColumnNames: _col0, _col2
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(val)
-                keys: key (type: string), val (type: string)
+                aggregations: count(1)
+                keys: _col0 (type: string), _col2 (type: string)
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1893,7 +1893,7 @@ STAGE PLANS:
               name: default.t1
             name: default.t1
       Truncated Path -> Alias:
-        /t1 [t1]
+        /t1 [$hdt$_0:t1]
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
@@ -2044,7 +2044,7 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@outputtbl4
-POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.null, ]
 POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl4.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ]
@@ -4585,11 +4585,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: key, val
+              outputColumnNames: _col0, _col2
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(val)
-                keys: key (type: string), val (type: string)
+                aggregations: count(1)
+                keys: _col0 (type: string), _col2 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4680,7 +4680,7 @@ STAGE PLANS:
               name: default.t2
             name: default.t2
       Truncated Path -> Alias:
-        /t2 [t2]
+        /t2 [$hdt$_0:t2]
 
   Stage: Stage-7
     Conditional Operator
@@ -4906,7 +4906,7 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t2
 POSTHOOK: Output: default@outputtbl4
-POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.null, ]
 POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl4.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
@@ -4964,11 +4964,11 @@ STAGE PLANS:
             GatherStats: false
             Select Operator
               expressions: key (type: string), val (type: string)
-              outputColumnNames: key, val
+              outputColumnNames: _col0, _col2
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                aggregations: count(val)
-                keys: key (type: string), val (type: string)
+                aggregations: count(1)
+                keys: _col0 (type: string), _col2 (type: string)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -5059,7 +5059,7 @@ STAGE PLANS:
               name: default.t2
             name: default.t2
       Truncated Path -> Alias:
-        /t2 [t2]
+        /t2 [$hdt$_0:t2]
 
   Stage: Stage-7
     Conditional Operator
@@ -5285,7 +5285,7 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t2
 POSTHOOK: Output: default@outputtbl5
-POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.null, ]
 POSTHOOK: Lineage: outputtbl5.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl5.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl5.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/limit_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/limit_pushdown.q.out b/ql/src/test/results/clientpositive/limit_pushdown.q.out
index 6aaf9b8..898662b 100644
--- a/ql/src/test/results/clientpositive/limit_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/limit_pushdown.q.out
@@ -695,7 +695,17 @@ STAGE PLANS:
     Fetch Operator
       limit: 0
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: key (type: string), value (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 0
+              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+              ListSink
 
 PREHOOK: query: select key,value from src order by key limit 0
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/limit_pushdown3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/limit_pushdown3.q.out b/ql/src/test/results/clientpositive/limit_pushdown3.q.out
index 8ccda6a..66dc77f 100644
--- a/ql/src/test/results/clientpositive/limit_pushdown3.q.out
+++ b/ql/src/test/results/clientpositive/limit_pushdown3.q.out
@@ -824,7 +824,17 @@ STAGE PLANS:
     Fetch Operator
       limit: 0
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: key (type: string), value (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 0
+              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+              ListSink
 
 PREHOOK: query: select key,value from src order by key limit 0
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_4.q.out b/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
index 4ea1488..0978ddd 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_4.q.out
@@ -28,11 +28,11 @@ Stage-0
     Stage-1
       Reducer 3 llap
       File Output Operator [FS_12]
-        Select Operator [SEL_11] (rows=7286 width=620)
+        Select Operator [SEL_11] (rows=9759 width=620)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
         <-Reducer 2 [SIMPLE_EDGE] llap
           SHUFFLE [RS_10]
-            Merge Join Operator [MERGEJOIN_17] (rows=7286 width=620)
+            Merge Join Operator [MERGEJOIN_17] (rows=9759 width=620)
               Conds:RS_6._col2=RS_7._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
             <-Map 1 [SIMPLE_EDGE] llap
               SHUFFLE [RS_6]
@@ -46,10 +46,10 @@ Stage-0
             <-Map 4 [SIMPLE_EDGE] llap
               SHUFFLE [RS_7]
                 PartitionCols:_col2
-                Select Operator [SEL_5] (rows=3424 width=251)
+                Select Operator [SEL_5] (rows=4586 width=251)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                  Filter Operator [FIL_16] (rows=3424 width=251)
-                    predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                  Filter Operator [FIL_16] (rows=4586 width=251)
+                    predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                     TableScan [TS_3] (rows=12288 width=251)
                       default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
 
@@ -113,7 +113,7 @@ Stage-0
           SHUFFLE [RS_11]
             Group By Operator [GBY_10] (rows=1 width=8)
               Output:["_col0"],aggregations:["count()"]
-              Merge Join Operator [MERGEJOIN_19] (rows=7286 width=8)
+              Merge Join Operator [MERGEJOIN_19] (rows=9759 width=8)
                 Conds:RS_6._col0=RS_7._col0(Inner)
               <-Map 1 [SIMPLE_EDGE] llap
                 SHUFFLE [RS_6]
@@ -127,10 +127,10 @@ Stage-0
               <-Map 4 [SIMPLE_EDGE] llap
                 SHUFFLE [RS_7]
                   PartitionCols:_col0
-                  Select Operator [SEL_5] (rows=3424 width=8)
+                  Select Operator [SEL_5] (rows=4586 width=8)
                     Output:["_col0"]
-                    Filter Operator [FIL_18] (rows=3424 width=8)
-                      predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                    Filter Operator [FIL_18] (rows=4586 width=8)
+                      predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                       TableScan [TS_3] (rows=12288 width=8)
                         default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["cint","cbigint"]
 
@@ -193,7 +193,7 @@ Stage-0
                 PartitionCols:_col0
                 Group By Operator [GBY_10] (rows=2765 width=12)
                   Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                  Merge Join Operator [MERGEJOIN_21] (rows=7286 width=4)
+                  Merge Join Operator [MERGEJOIN_21] (rows=9759 width=4)
                     Conds:RS_6._col1=RS_7._col0(Inner),Output:["_col0"]
                   <-Map 1 [SIMPLE_EDGE] llap
                     SHUFFLE [RS_6]
@@ -207,10 +207,10 @@ Stage-0
                   <-Map 5 [SIMPLE_EDGE] llap
                     SHUFFLE [RS_7]
                       PartitionCols:_col0
-                      Select Operator [SEL_5] (rows=3424 width=8)
+                      Select Operator [SEL_5] (rows=4586 width=8)
                         Output:["_col0"]
-                        Filter Operator [FIL_20] (rows=3424 width=8)
-                          predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                        Filter Operator [FIL_20] (rows=4586 width=8)
+                          predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                           TableScan [TS_3] (rows=12288 width=8)
                             default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["cint","cbigint"]
 
@@ -281,7 +281,7 @@ Stage-0
                 Select Operator [SEL_5] (rows=6144 width=215)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                   Filter Operator [FIL_16] (rows=6144 width=215)
-                    predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                    predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                     TableScan [TS_3] (rows=12288 width=215)
                       default@alltypesorc,b,Tbl:COMPLETE,Col:NONE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
             <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
@@ -362,7 +362,7 @@ Stage-0
                   Select Operator [SEL_5] (rows=6144 width=215)
                     Output:["_col0"]
                     Filter Operator [FIL_18] (rows=6144 width=215)
-                      predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                      predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                       TableScan [TS_3] (rows=12288 width=215)
                         default@alltypesorc,b,Tbl:COMPLETE,Col:NONE,Output:["cint","cbigint"]
               <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
@@ -442,7 +442,7 @@ Stage-0
                       Select Operator [SEL_5] (rows=6144 width=215)
                         Output:["_col0"]
                         Filter Operator [FIL_20] (rows=6144 width=215)
-                          predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                          predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                           TableScan [TS_3] (rows=12288 width=215)
                             default@alltypesorc,b,Tbl:COMPLETE,Col:NONE,Output:["cint","cbigint"]
                   <-Map 1 [CUSTOM_SIMPLE_EDGE] llap

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out b/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
index 3fe4837..507791e 100644
--- a/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
@@ -778,7 +778,14 @@ STAGE PLANS:
     Fetch Operator
       limit: 0
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Select Operator
+            expressions: key (type: string), value (type: string)
+            outputColumnNames: _col0, _col1
+            Limit
+              Number of rows: 0
+              ListSink
 
 PREHOOK: query: select key,value from src order by key limit 0
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/llap/lineage3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/lineage3.q.out b/ql/src/test/results/clientpositive/llap/lineage3.q.out
index 257c547..1a532da 100644
--- a/ql/src/test/results/clientpositive/llap/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/llap/lineage3.q.out
@@ -116,7 +116,7 @@ order by a.cbigint, a.ctinyint, b.cint, b.ctinyint limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-{"version":"1.0","engine":"tez","database":"default","hash":"afd760470fc5aa6d3e8348dee03af97f","queryText":"select a.cbigint, a.ctinyint, b.cint, b.ctinyint\nfrom\n  (select ctinyint, cbigint from alltypesorc\n   union all\n   select ctinyint, cbigint from alltypesorc) a\n  inner join\n  alltypesorc b\n  on (a.ctinyint = b.ctinyint)\nwhere b.ctinyint < 100 and a.cbigint is not null and b.cint is not null\norder by a.cbigint, a.ctinyint, b.cint, b.ctinyint limit 5","edges":[{"sources":[4],"targets":[0],"expression":"cbigint","edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"ctinyint","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[5],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5,4],"targets":[0,1,2,3],"expression":"(alltypesorc.ctinyint is not null and alltypesorc.cbigint is not null and (alltypesorc.ctinyint < 100))","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(ctinyint = b.ctinyint)","e
 dgeType":"PREDICATE"},{"sources":[5,6],"targets":[0,1,2,3],"expression":"((b.ctinyint < 100) and b.cint is not null)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.cbigint"},{"id":1,"vertexType":"COLUMN","vertexId":"a.ctinyint"},{"id":2,"vertexType":"COLUMN","vertexId":"b.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"b.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"}]}
+{"version":"1.0","engine":"tez","database":"default","hash":"afd760470fc5aa6d3e8348dee03af97f","queryText":"select a.cbigint, a.ctinyint, b.cint, b.ctinyint\nfrom\n  (select ctinyint, cbigint from alltypesorc\n   union all\n   select ctinyint, cbigint from alltypesorc) a\n  inner join\n  alltypesorc b\n  on (a.ctinyint = b.ctinyint)\nwhere b.ctinyint < 100 and a.cbigint is not null and b.cint is not null\norder by a.cbigint, a.ctinyint, b.cint, b.ctinyint limit 5","edges":[{"sources":[4],"targets":[0],"expression":"cbigint","edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"ctinyint","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[5],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5,4],"targets":[0,1,2,3],"expression":"((alltypesorc.ctinyint < 100) and alltypesorc.cbigint is not null)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(ctinyint = b.ctinyint)","edgeType":"PREDICATE"},{"sources":[5,6
 ],"targets":[0,1,2,3],"expression":"((b.ctinyint < 100) and b.cint is not null)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.cbigint"},{"id":1,"vertexType":"COLUMN","vertexId":"a.ctinyint"},{"id":2,"vertexType":"COLUMN","vertexId":"b.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"b.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"}]}
 -2147311592	-51	-1071480828	-51
 -2147311592	-51	-1071480828	-51
 -2147311592	-51	-1067683781	-51

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/llap/table_access_keys_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/table_access_keys_stats.q.out b/ql/src/test/results/clientpositive/llap/table_access_keys_stats.q.out
index 91bdff3..e68f60f 100644
--- a/ql/src/test/results/clientpositive/llap/table_access_keys_stats.q.out
+++ b/ql/src/test/results/clientpositive/llap/table_access_keys_stats.q.out
@@ -77,7 +77,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1
 #### A masked pattern was here ####
-Operator:GBY_2
+Operator:GBY_3
 Table:default@t1
 Keys:key
 
@@ -90,7 +90,7 @@ PREHOOK: query: SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1
 #### A masked pattern was here ####
-Operator:GBY_2
+Operator:GBY_3
 Table:default@t1
 Keys:key,val
 
@@ -104,7 +104,7 @@ PREHOOK: query: SELECT key, 1, val, 2, count(1) FROM T1 GROUP BY key, 1, val, 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1
 #### A masked pattern was here ####
-Operator:GBY_2
+Operator:GBY_3
 Table:default@t1
 Keys:key,val
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
index 3c6ef9a..5c8db64 100644
--- a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_1.q.out
@@ -55,17 +55,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
-                    Statistics: Num rows: 3424 Data size: 862030 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
+                    Statistics: Num rows: 4586 Data size: 1154510 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                      Statistics: Num rows: 3424 Data size: 862030 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4586 Data size: 1154510 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col2 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col2 (type: int)
-                        Statistics: Num rows: 3424 Data size: 862030 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4586 Data size: 1154510 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: llap
             LLAP IO: all inputs
@@ -79,11 +79,11 @@ STAGE PLANS:
                   0 _col2 (type: int)
                   1 _col2 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
-                Statistics: Num rows: 7286 Data size: 4517320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9759 Data size: 6050580 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col2 (type: int)
                   sort order: +
-                  Statistics: Num rows: 7286 Data size: 4517320 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9759 Data size: 6050580 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean), _col12 (type: tinyint), _col13 (type: smallint), _col14 (type: int), _col15 (type: bigint), _col16 (type: float), _col17 (type: double), _col18 (type: string), _col19 (type: string), _col20 (type: timestamp), _col21 (type: timestamp), _col22 (type: boolean), _col23 (type: boolean)
         Reducer 3 
             Execution mode: llap
@@ -91,10 +91,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: timestamp), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), VALUE._col11 (type: tinyint), VALUE._col12 (type: smallint), VALUE._col13 (type: int), VALUE._col14 (type: bigint), VALUE._col15 (type: float), VALUE._col16 (type: double), VALUE._col17 (type: string), VALUE._col18 (type: string), VALUE._col19 (type: timestamp), VALUE._col20 (type: timestamp), VALUE._col21 (type: boolean), VALUE._col22 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
-                Statistics: Num rows: 7286 Data size: 4517320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9759 Data size: 6050580 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 7286 Data size: 4517320 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9759 Data size: 6050580 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -186,17 +186,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 110088 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
-                    Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
+                    Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -208,7 +208,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 7286 Data size: 58288 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9759 Data size: 78072 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -315,17 +315,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 110088 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
-                    Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
+                    Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -338,7 +338,7 @@ STAGE PLANS:
                   0 _col1 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 7286 Data size: 29144 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9759 Data size: 39036 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: smallint)
@@ -468,7 +468,7 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
                     Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
@@ -602,7 +602,7 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
                     Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cint (type: int)
@@ -734,7 +734,7 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
                     Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cint (type: int)

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
index c3aebc7..b2359c1 100644
--- a/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_vector_dynpart_hashjoin_1.q.out
@@ -55,17 +55,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 3093170 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
-                    Statistics: Num rows: 3424 Data size: 862030 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
+                    Statistics: Num rows: 4586 Data size: 1154510 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                      Statistics: Num rows: 3424 Data size: 862030 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4586 Data size: 1154510 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col2 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col2 (type: int)
-                        Statistics: Num rows: 3424 Data size: 862030 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4586 Data size: 1154510 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
             Execution mode: llap
             LLAP IO: all inputs
@@ -79,11 +79,11 @@ STAGE PLANS:
                   0 _col2 (type: int)
                   1 _col2 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
-                Statistics: Num rows: 7286 Data size: 4517320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9759 Data size: 6050580 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col2 (type: int)
                   sort order: +
-                  Statistics: Num rows: 7286 Data size: 4517320 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9759 Data size: 6050580 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean), _col12 (type: tinyint), _col13 (type: smallint), _col14 (type: int), _col15 (type: bigint), _col16 (type: float), _col17 (type: double), _col18 (type: string), _col19 (type: string), _col20 (type: timestamp), _col21 (type: timestamp), _col22 (type: boolean), _col23 (type: boolean)
         Reducer 3 
             Execution mode: llap
@@ -91,10 +91,10 @@ STAGE PLANS:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: timestamp), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), VALUE._col11 (type: tinyint), VALUE._col12 (type: smallint), VALUE._col13 (type: int), VALUE._col14 (type: bigint), VALUE._col15 (type: float), VALUE._col16 (type: double), VALUE._col17 (type: string), VALUE._col18 (type: string), VALUE._col19 (type: timestamp), VALUE._col20 (type: timestamp), VALUE._col21 (type: boolean), VALUE._col22 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
-                Statistics: Num rows: 7286 Data size: 4517320 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9759 Data size: 6050580 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 7286 Data size: 4517320 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9759 Data size: 6050580 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -186,17 +186,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 110088 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
-                    Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
+                    Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -208,7 +208,7 @@ STAGE PLANS:
                 keys:
                   0 _col0 (type: int)
                   1 _col0 (type: int)
-                Statistics: Num rows: 7286 Data size: 58288 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9759 Data size: 78072 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   mode: hash
@@ -315,17 +315,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 110088 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
-                    Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
+                    Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: cint (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
-                        Statistics: Num rows: 3424 Data size: 30684 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 4586 Data size: 41088 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
@@ -338,7 +338,7 @@ STAGE PLANS:
                   0 _col1 (type: int)
                   1 _col0 (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 7286 Data size: 29144 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9759 Data size: 39036 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
                   aggregations: count()
                   keys: _col0 (type: smallint)
@@ -468,7 +468,7 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
                     Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
@@ -602,7 +602,7 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
                     Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cint (type: int)
@@ -734,7 +734,7 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000) (type: boolean)
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cbigint is not null) (type: boolean)
                     Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: cint (type: int)

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/offset_limit_ppd_optimizer.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/offset_limit_ppd_optimizer.q.out b/ql/src/test/results/clientpositive/offset_limit_ppd_optimizer.q.out
index 14cde78..71766a7 100644
--- a/ql/src/test/results/clientpositive/offset_limit_ppd_optimizer.q.out
+++ b/ql/src/test/results/clientpositive/offset_limit_ppd_optimizer.q.out
@@ -695,7 +695,17 @@ STAGE PLANS:
     Fetch Operator
       limit: 0
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: key (type: string), value (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Limit
+              Number of rows: 0
+              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+              ListSink
 
 PREHOOK: query: select key,value from src order by key limit 0,0
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/perf/query75.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query75.q.out b/ql/src/test/results/clientpositive/perf/query75.q.out
index 0c72248..f78ce8a 100644
--- a/ql/src/test/results/clientpositive/perf/query75.q.out
+++ b/ql/src/test/results/clientpositive/perf/query75.q.out
@@ -81,7 +81,7 @@ Stage-0
                                         Select Operator [SEL_82] (rows=231000 width=1436)
                                           Output:["_col0","_col1","_col2","_col3","_col5"]
                                           Filter Operator [FIL_231] (rows=231000 width=1436)
-                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null)
+                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
                                             TableScan [TS_80] (rows=462000 width=1436)
                                               default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
                                     <-Reducer 27 [SIMPLE_EDGE]
@@ -136,7 +136,7 @@ Stage-0
                                         Select Operator [SEL_104] (rows=231000 width=1436)
                                           Output:["_col0","_col1","_col2","_col3","_col5"]
                                           Filter Operator [FIL_235] (rows=231000 width=1436)
-                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null)
+                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
                                             TableScan [TS_102] (rows=462000 width=1436)
                                               default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
                                     <-Reducer 36 [SIMPLE_EDGE]
@@ -191,7 +191,7 @@ Stage-0
                                         Select Operator [SEL_128] (rows=231000 width=1436)
                                           Output:["_col0","_col1","_col2","_col3","_col5"]
                                           Filter Operator [FIL_239] (rows=231000 width=1436)
-                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null)
+                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
                                             TableScan [TS_126] (rows=462000 width=1436)
                                               default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
                                     <-Reducer 43 [SIMPLE_EDGE]
@@ -252,7 +252,7 @@ Stage-0
                                         Select Operator [SEL_30] (rows=231000 width=1436)
                                           Output:["_col0","_col1","_col2","_col3","_col5"]
                                           Filter Operator [FIL_223] (rows=231000 width=1436)
-                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null)
+                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
                                             TableScan [TS_28] (rows=462000 width=1436)
                                               default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
                                     <-Reducer 13 [SIMPLE_EDGE]
@@ -307,7 +307,7 @@ Stage-0
                                         Select Operator [SEL_54] (rows=231000 width=1436)
                                           Output:["_col0","_col1","_col2","_col3","_col5"]
                                           Filter Operator [FIL_227] (rows=231000 width=1436)
-                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null)
+                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
                                             TableScan [TS_52] (rows=462000 width=1436)
                                               default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
                                     <-Reducer 20 [SIMPLE_EDGE]
@@ -362,7 +362,7 @@ Stage-0
                                         Select Operator [SEL_8] (rows=231000 width=1436)
                                           Output:["_col0","_col1","_col2","_col3","_col5"]
                                           Filter Operator [FIL_219] (rows=231000 width=1436)
-                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null)
+                                            predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null)
                                             TableScan [TS_6] (rows=462000 width=1436)
                                               default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"]
                                     <-Reducer 2 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out b/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out
index c6a7982..92ca67b 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out
@@ -836,11 +836,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: key
+                    outputColumnNames: _col1
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(key)
-                      keys: key (type: string)
+                      aggregations: count(1)
+                      keys: _col1 (type: string)
                       mode: final
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -974,7 +974,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@outputtbl3
-POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.null, ]
 POSTHOOK: Lineage: outputtbl3.key1 SIMPLE []
 POSTHOOK: Lineage: outputtbl3.key2 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
 PREHOOK: query: SELECT * FROM outputTbl3
@@ -1028,11 +1028,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: key, val
+                    outputColumnNames: _col0, _col2
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(val)
-                      keys: key (type: string), val (type: string)
+                      aggregations: count(1)
+                      keys: _col0 (type: string), _col2 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1184,7 +1184,7 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@outputtbl4
-POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.null, ]
 POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl4.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ]
@@ -3074,11 +3074,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: key, val
+                    outputColumnNames: _col0, _col2
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(val)
-                      keys: key (type: string), val (type: string)
+                      aggregations: count(1)
+                      keys: _col0 (type: string), _col2 (type: string)
                       mode: final
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3212,7 +3212,7 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t2
 POSTHOOK: Output: default@outputtbl4
-POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.null, ]
 POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl4.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
@@ -3268,11 +3268,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: key, val
+                    outputColumnNames: _col0, _col2
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(val)
-                      keys: key (type: string), val (type: string)
+                      aggregations: count(1)
+                      keys: _col0 (type: string), _col2 (type: string)
                       mode: final
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3406,7 +3406,7 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t2
 POSTHOOK: Output: default@outputtbl5
-POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.null, ]
 POSTHOOK: Lineage: outputtbl5.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl5.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl5.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out b/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out
index a438124..f7f4dbb 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out
@@ -855,11 +855,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string)
-                    outputColumnNames: key
+                    outputColumnNames: _col1
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(key)
-                      keys: key (type: string)
+                      aggregations: count(1)
+                      keys: _col1 (type: string)
                       mode: final
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -993,7 +993,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@outputtbl3
-POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.null, ]
 POSTHOOK: Lineage: outputtbl3.key1 SIMPLE []
 POSTHOOK: Lineage: outputtbl3.key2 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
 PREHOOK: query: SELECT * FROM outputTbl3
@@ -1048,11 +1048,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: key, val
+                    outputColumnNames: _col0, _col2
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(val)
-                      keys: key (type: string), val (type: string)
+                      aggregations: count(1)
+                      keys: _col0 (type: string), _col2 (type: string)
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1222,7 +1222,7 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t1
 POSTHOOK: Output: default@outputtbl4
-POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.null, ]
 POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl4.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ]
@@ -3207,11 +3207,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: key, val
+                    outputColumnNames: _col0, _col2
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(val)
-                      keys: key (type: string), val (type: string)
+                      aggregations: count(1)
+                      keys: _col0 (type: string), _col2 (type: string)
                       mode: final
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3345,7 +3345,7 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t2
 POSTHOOK: Output: default@outputtbl4
-POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.null, ]
 POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl4.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
@@ -3401,11 +3401,11 @@ STAGE PLANS:
                   GatherStats: false
                   Select Operator
                     expressions: key (type: string), val (type: string)
-                    outputColumnNames: key, val
+                    outputColumnNames: _col0, _col2
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: count(val)
-                      keys: key (type: string), val (type: string)
+                      aggregations: count(1)
+                      keys: _col0 (type: string), _col2 (type: string)
                       mode: final
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3539,7 +3539,7 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t2
 POSTHOOK: Output: default@outputtbl5
-POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.null, ]
 POSTHOOK: Lineage: outputtbl5.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: outputtbl5.key2 SIMPLE []
 POSTHOOK: Lineage: outputtbl5.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ]

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out b/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out
index 67c6e70..95de5b1 100644
--- a/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/spark/limit_pushdown.q.out
@@ -746,7 +746,14 @@ STAGE PLANS:
     Fetch Operator
       limit: 0
       Processor Tree:
-        ListSink
+        TableScan
+          alias: src
+          Select Operator
+            expressions: key (type: string), value (type: string)
+            outputColumnNames: _col0, _col1
+            Limit
+              Number of rows: 0
+              ListSink
 
 PREHOOK: query: select key,value from src order by key limit 0
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out b/ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out
index e26ccec..07727d4 100644
--- a/ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out
+++ b/ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out
@@ -77,7 +77,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1
 #### A masked pattern was here ####
-Operator:GBY_2
+Operator:GBY_3
 Table:default@t1
 Keys:key
 
@@ -90,7 +90,7 @@ PREHOOK: query: SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1
 #### A masked pattern was here ####
-Operator:GBY_2
+Operator:GBY_3
 Table:default@t1
 Keys:key,val
 
@@ -104,7 +104,7 @@ PREHOOK: query: SELECT key, 1, val, 2, count(1) FROM T1 GROUP BY key, 1, val, 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1
 #### A masked pattern was here ####
-Operator:GBY_2
+Operator:GBY_3
 Table:default@t1
 Keys:key,val
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b597ab2a/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
index 3426d19..9a3d59f 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_4.q.out
@@ -46,11 +46,11 @@ Stage-0
     Stage-1
       Reducer 3
       File Output Operator [FS_12]
-        Select Operator [SEL_11] (rows=7286/10 width=620)
+        Select Operator [SEL_11] (rows=9759/10 width=620)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
         <-Reducer 2 [SIMPLE_EDGE]
           SHUFFLE [RS_10]
-            Merge Join Operator [MERGEJOIN_17] (rows=7286/10 width=620)
+            Merge Join Operator [MERGEJOIN_17] (rows=9759/10 width=620)
               Conds:RS_6._col2=RS_7._col2(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23"]
             <-Map 1 [SIMPLE_EDGE]
               SHUFFLE [RS_6]
@@ -64,10 +64,10 @@ Stage-0
             <-Map 4 [SIMPLE_EDGE]
               SHUFFLE [RS_7]
                 PartitionCols:_col2
-                Select Operator [SEL_5] (rows=3424/10 width=251)
+                Select Operator [SEL_5] (rows=4586/10 width=251)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
-                  Filter Operator [FIL_16] (rows=3424/10 width=251)
-                    predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                  Filter Operator [FIL_16] (rows=4586/10 width=251)
+                    predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                     TableScan [TS_3] (rows=12288/12288 width=251)
                       default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
 
@@ -147,7 +147,7 @@ Stage-0
           SHUFFLE [RS_11]
             Group By Operator [GBY_10] (rows=1/1 width=8)
               Output:["_col0"],aggregations:["count()"]
-              Merge Join Operator [MERGEJOIN_19] (rows=7286/10 width=8)
+              Merge Join Operator [MERGEJOIN_19] (rows=9759/10 width=8)
                 Conds:RS_6._col0=RS_7._col0(Inner)
               <-Map 1 [SIMPLE_EDGE]
                 SHUFFLE [RS_6]
@@ -161,10 +161,10 @@ Stage-0
               <-Map 4 [SIMPLE_EDGE]
                 SHUFFLE [RS_7]
                   PartitionCols:_col0
-                  Select Operator [SEL_5] (rows=3424/10 width=8)
+                  Select Operator [SEL_5] (rows=4586/10 width=8)
                     Output:["_col0"]
-                    Filter Operator [FIL_18] (rows=3424/10 width=8)
-                      predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                    Filter Operator [FIL_18] (rows=4586/10 width=8)
+                      predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                       TableScan [TS_3] (rows=12288/12288 width=8)
                         default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["cint","cbigint"]
 
@@ -247,7 +247,7 @@ Stage-0
                 PartitionCols:_col0
                 Group By Operator [GBY_10] (rows=2765/5 width=12)
                   Output:["_col0","_col1"],aggregations:["count()"],keys:_col0
-                  Merge Join Operator [MERGEJOIN_21] (rows=7286/10 width=4)
+                  Merge Join Operator [MERGEJOIN_21] (rows=9759/10 width=4)
                     Conds:RS_6._col1=RS_7._col0(Inner),Output:["_col0"]
                   <-Map 1 [SIMPLE_EDGE]
                     SHUFFLE [RS_6]
@@ -261,10 +261,10 @@ Stage-0
                   <-Map 5 [SIMPLE_EDGE]
                     SHUFFLE [RS_7]
                       PartitionCols:_col0
-                      Select Operator [SEL_5] (rows=3424/10 width=8)
+                      Select Operator [SEL_5] (rows=4586/10 width=8)
                         Output:["_col0"]
-                        Filter Operator [FIL_20] (rows=3424/10 width=8)
-                          predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                        Filter Operator [FIL_20] (rows=4586/10 width=8)
+                          predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                           TableScan [TS_3] (rows=12288/12288 width=8)
                             default@alltypesorc,b,Tbl:COMPLETE,Col:COMPLETE,Output:["cint","cbigint"]
 
@@ -353,7 +353,7 @@ Stage-0
                 Select Operator [SEL_5] (rows=6144/10 width=215)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11"]
                   Filter Operator [FIL_16] (rows=6144/10 width=215)
-                    predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                    predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                     TableScan [TS_3] (rows=12288/12288 width=215)
                       default@alltypesorc,b,Tbl:COMPLETE,Col:NONE,Output:["ctinyint","csmallint","cint","cbigint","cfloat","cdouble","cstring1","cstring2","ctimestamp1","ctimestamp2","cboolean1","cboolean2"]
             <-Map 1 [CUSTOM_SIMPLE_EDGE]
@@ -450,7 +450,7 @@ Stage-0
                   Select Operator [SEL_5] (rows=6144/10 width=215)
                     Output:["_col0"]
                     Filter Operator [FIL_18] (rows=6144/10 width=215)
-                      predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                      predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                       TableScan [TS_3] (rows=12288/12288 width=215)
                         default@alltypesorc,b,Tbl:COMPLETE,Col:NONE,Output:["cint","cbigint"]
               <-Map 1 [CUSTOM_SIMPLE_EDGE]
@@ -550,7 +550,7 @@ Stage-0
                       Select Operator [SEL_5] (rows=6144/10 width=215)
                         Output:["_col0"]
                         Filter Operator [FIL_20] (rows=6144/10 width=215)
-                          predicate:(cint is not null and cbigint is not null and cint BETWEEN 1000000 AND 3000000)
+                          predicate:(cint BETWEEN 1000000 AND 3000000 and cbigint is not null)
                           TableScan [TS_3] (rows=12288/12288 width=215)
                             default@alltypesorc,b,Tbl:COMPLETE,Col:NONE,Output:["cint","cbigint"]
                   <-Map 1 [CUSTOM_SIMPLE_EDGE]