You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/09/04 04:55:15 UTC

[26/28] hive git commit: HIVE-11383: Upgrade Hive to Calcite 1.4 (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

HIVE-11383: Upgrade Hive to Calcite 1.4 (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/c40382d4
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c40382d4
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c40382d4

Branch: refs/heads/llap
Commit: c40382d4c3bd3f0b0c0f1b09affb7d03198f47ba
Parents: 492c8b1
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Thu Sep 3 11:48:07 2015 +0200
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Sep 3 11:48:07 2015 +0200

----------------------------------------------------------------------
 pom.xml                                         |   2 +-
 .../ql/optimizer/calcite/HiveRelOptUtil.java    |  23 --
 .../calcite/reloperators/HiveSort.java          |  29 +-
 .../rules/HiveJoinProjectTransposeRule.java     | 238 ++--------------
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   5 +-
 .../clientpositive/constprog_partitioner.q.out  |  30 +-
 .../clientpositive/correlationoptimizer10.q.out |  48 ++--
 .../spark/constprog_partitioner.q.out           |  30 +-
 .../clientpositive/spark/subquery_exists.q.out  |  12 +-
 .../clientpositive/spark/subquery_in.q.out      |  36 +--
 .../spark/vector_mapjoin_reduce.q.out           |  22 +-
 .../clientpositive/subquery_exists.q.out        |  12 +-
 .../results/clientpositive/subquery_in.q.out    |  36 +--
 .../clientpositive/subquery_in_having.q.out     |  50 ++--
 .../subquery_unqualcolumnrefs.q.out             |  26 +-
 .../results/clientpositive/subquery_views.q.out |  40 +--
 .../clientpositive/tez/explainuser_1.q.out      | 278 +++++++++----------
 .../clientpositive/tez/subquery_exists.q.out    |  12 +-
 .../clientpositive/tez/subquery_in.q.out        |  36 +--
 .../clientpositive/tez/vector_inner_join.q.out  |  14 +-
 .../tez/vector_mapjoin_reduce.q.out             |  24 +-
 .../clientpositive/vector_inner_join.q.out      |  12 +-
 .../clientpositive/vector_mapjoin_reduce.q.out  |  26 +-
 23 files changed, 431 insertions(+), 610 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 15c2805..b55e86a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -103,7 +103,7 @@
     <antlr.version>3.4</antlr.version>
     <avro.version>1.7.7</avro.version>
     <bonecp.version>0.8.0.RELEASE</bonecp.version>
-    <calcite.version>1.3.0-incubating</calcite.version>
+    <calcite.version>1.4.0-incubating</calcite.version>
     <datanucleus-api-jdo.version>3.2.6</datanucleus-api-jdo.version>
     <datanucleus-core.version>3.2.10</datanucleus-core.version>
     <datanucleus-rdbms.version>3.2.9</datanucleus-rdbms.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/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 5a5954d..0e282b8 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
@@ -278,29 +278,6 @@ public class HiveRelOptUtil extends RelOptUtil {
     }
   }
 
-  private static SqlOperator op(SqlKind kind, SqlOperator operator) {
-    switch (kind) {
-    case EQUALS:
-      return SqlStdOperatorTable.EQUALS;
-    case NOT_EQUALS:
-      return SqlStdOperatorTable.NOT_EQUALS;
-    case GREATER_THAN:
-      return SqlStdOperatorTable.GREATER_THAN;
-    case GREATER_THAN_OR_EQUAL:
-      return SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
-    case LESS_THAN:
-      return SqlStdOperatorTable.LESS_THAN;
-    case LESS_THAN_OR_EQUAL:
-      return SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
-    case IS_DISTINCT_FROM:
-      return SqlStdOperatorTable.IS_DISTINCT_FROM;
-    case IS_NOT_DISTINCT_FROM:
-      return SqlStdOperatorTable.IS_NOT_DISTINCT_FROM;
-    default:
-      return operator;
-    }
-  }
-
   private static void addJoinKey(
       List<RexNode> joinKeyList,
       RexNode key,

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java
index 18d2838..1df6542 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSort.java
@@ -22,6 +22,7 @@ import java.util.Map;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Sort;
@@ -49,6 +50,24 @@ public class HiveSort extends Sort implements HiveRelNode {
         offset, fetch);
   }
 
+  /**
+   * Creates a HiveSort.
+   *
+   * @param input     Input relational expression
+   * @param collation array of sort specifications
+   * @param offset    Expression for number of rows to discard before returning
+   *                  first row
+   * @param fetch     Expression for number of rows to fetch
+   */
+  public static HiveSort create(RelNode input, RelCollation collation,
+      RexNode offset, RexNode fetch) {
+    RelOptCluster cluster = input.getCluster();
+    collation = RelCollationTraitDef.INSTANCE.canonize(collation);
+    RelTraitSet traitSet =
+        TraitsUtil.getSortTraitSet(cluster, input.getTraitSet(), collation);
+    return new HiveSort(cluster, traitSet, input, collation, offset, fetch);
+  }
+
   @Override
   public HiveSort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation,
       RexNode offset, RexNode fetch) {
@@ -77,9 +96,15 @@ public class HiveSort extends Sort implements HiveRelNode {
   private static class HiveSortRelFactory implements RelFactories.SortFactory {
 
     @Override
-    public RelNode createSort(RelTraitSet traits, RelNode child, RelCollation collation,
+    public RelNode createSort(RelTraitSet traits, RelNode input, RelCollation collation,
         RexNode offset, RexNode fetch) {
-      return new HiveSort(child.getCluster(), traits, child, collation, offset, fetch);
+      return createSort(input, collation, offset, fetch);
+    }
+
+    @Override
+    public RelNode createSort(RelNode input, RelCollation collation, RexNode offset,
+        RexNode fetch) {
+      return create(input, collation, offset, fetch);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
index fd8f5cb..ac72ee5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
@@ -49,14 +49,14 @@ public class HiveJoinProjectTransposeRule extends JoinProjectTransposeRule {
               operand(HiveProject.class, any()),
               operand(HiveProject.class, any())),
           "JoinProjectTransposeRule(Project-Project)",
-          HiveProject.DEFAULT_PROJECT_FACTORY);
+          false, HiveProject.DEFAULT_PROJECT_FACTORY);
 
   public static final HiveJoinProjectTransposeRule LEFT_PROJECT =
       new HiveJoinProjectTransposeRule(
           operand(HiveJoin.class,
               some(operand(HiveProject.class, any()))),
           "JoinProjectTransposeRule(Project-Other)",
-          HiveProject.DEFAULT_PROJECT_FACTORY);
+          false, HiveProject.DEFAULT_PROJECT_FACTORY);
 
   public static final HiveJoinProjectTransposeRule RIGHT_PROJECT =
       new HiveJoinProjectTransposeRule(
@@ -65,219 +65,37 @@ public class HiveJoinProjectTransposeRule extends JoinProjectTransposeRule {
               operand(RelNode.class, any()),
               operand(HiveProject.class, any())),
           "JoinProjectTransposeRule(Other-Project)",
-          HiveProject.DEFAULT_PROJECT_FACTORY);
+          false, HiveProject.DEFAULT_PROJECT_FACTORY);
 
-  private final ProjectFactory projectFactory;
-
-
-  private HiveJoinProjectTransposeRule(
-      RelOptRuleOperand operand,
-      String description, ProjectFactory pFactory) {
-    super(operand, description, pFactory);
-    this.projectFactory = pFactory;
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    Join joinRel = call.rel(0);
-    JoinRelType joinType = joinRel.getJoinType();
-
-    Project leftProj;
-    Project rightProj;
-    RelNode leftJoinChild;
-    RelNode rightJoinChild;
-
-    // see if at least one input's projection doesn't generate nulls
-    if (hasLeftChild(call)) {
-      leftProj = call.rel(1);
-      leftJoinChild = getProjectChild(call, leftProj, true);
-    } else {
-      leftProj = null;
-      leftJoinChild = call.rel(1);
-    }
-    if (hasRightChild(call)) {
-      rightProj = getRightChild(call);
-      rightJoinChild = getProjectChild(call, rightProj, false);
-    } else {
-      rightProj = null;
-      rightJoinChild = joinRel.getRight();
-    }
-    if ((leftProj == null) && (rightProj == null)) {
-      return;
-    }
-
-    // Construct two RexPrograms and combine them.  The bottom program
-    // is a join of the projection expressions from the left and/or
-    // right projects that feed into the join.  The top program contains
-    // the join condition.
-
-    // Create a row type representing a concatenation of the inputs
-    // underneath the projects that feed into the join.  This is the input
-    // into the bottom RexProgram.  Note that the join type is an inner
-    // join because the inputs haven't actually been joined yet.
-    RelDataType joinChildrenRowType =
-        Join.deriveJoinRowType(
-            leftJoinChild.getRowType(),
-            rightJoinChild.getRowType(),
-            JoinRelType.INNER,
-            joinRel.getCluster().getTypeFactory(),
-            null,
-            Collections.<RelDataTypeField>emptyList());
-
-    // Create projection expressions, combining the projection expressions
-    // from the projects that feed into the join.  For the RHS projection
-    // expressions, shift them to the right by the number of fields on
-    // the LHS.  If the join input was not a projection, simply create
-    // references to the inputs.
-    int nProjExprs = joinRel.getRowType().getFieldCount();
-    List<Pair<RexNode, String>> projects =
-        new ArrayList<Pair<RexNode, String>>();
-    RexBuilder rexBuilder = joinRel.getCluster().getRexBuilder();
-
-    createProjectExprs(
-        leftProj,
-        leftJoinChild,
-        0,
-        rexBuilder,
-        joinChildrenRowType.getFieldList(),
-        projects);
-
-    List<RelDataTypeField> leftFields =
-        leftJoinChild.getRowType().getFieldList();
-    int nFieldsLeft = leftFields.size();
-    createProjectExprs(
-        rightProj,
-        rightJoinChild,
-        nFieldsLeft,
-        rexBuilder,
-        joinChildrenRowType.getFieldList(),
-        projects);
-
-    List<RelDataType> projTypes = new ArrayList<RelDataType>();
-    for (int i = 0; i < nProjExprs; i++) {
-      projTypes.add(projects.get(i).left.getType());
-    }
-    RelDataType projRowType =
-        rexBuilder.getTypeFactory().createStructType(
-            projTypes,
-            Pair.right(projects));
-
-    // create the RexPrograms and merge them
-    RexProgram bottomProgram =
-        RexProgram.create(
-            joinChildrenRowType,
-            Pair.left(projects),
-            null,
-            projRowType,
-            rexBuilder);
-    RexProgramBuilder topProgramBuilder =
-        new RexProgramBuilder(
-            projRowType,
-            rexBuilder);
-    topProgramBuilder.addIdentity();
-    topProgramBuilder.addCondition(joinRel.getCondition());
-    RexProgram topProgram = topProgramBuilder.getProgram();
-    RexProgram mergedProgram =
-        RexProgramBuilder.mergePrograms(
-            topProgram,
-            bottomProgram,
-            rexBuilder);
+  public static final HiveJoinProjectTransposeRule BOTH_PROJECT_INCLUDE_OUTER =
+      new HiveJoinProjectTransposeRule(
+          operand(HiveJoin.class,
+              operand(HiveProject.class, any()),
+              operand(HiveProject.class, any())),
+          "Join(IncludingOuter)ProjectTransposeRule(Project-Project)",
+          true, HiveProject.DEFAULT_PROJECT_FACTORY);
 
-    // expand out the join condition and construct a new LogicalJoin that
-    // directly references the join children without the intervening
-    // ProjectRels
-    RexNode newCondition =
-        mergedProgram.expandLocalRef(
-            mergedProgram.getCondition());
-    Join newJoinRel =
-        joinRel.copy(joinRel.getTraitSet(), newCondition,
-            leftJoinChild, rightJoinChild, joinRel.getJoinType(),
-            joinRel.isSemiJoinDone());
+  public static final HiveJoinProjectTransposeRule LEFT_PROJECT_INCLUDE_OUTER =
+      new HiveJoinProjectTransposeRule(
+          operand(HiveJoin.class,
+              some(operand(HiveProject.class, any()))),
+          "Join(IncludingOuter)ProjectTransposeRule(Project-Other)",
+          true, HiveProject.DEFAULT_PROJECT_FACTORY);
 
-    // expand out the new projection expressions; if the join is an
-    // outer join, modify the expressions to reference the join output
-    List<RexNode> newProjExprs = new ArrayList<RexNode>();
-    List<RexLocalRef> projList = mergedProgram.getProjectList();
-    List<RelDataTypeField> newJoinFields =
-        newJoinRel.getRowType().getFieldList();
-    int nJoinFields = newJoinFields.size();
-    int[] adjustments = new int[nJoinFields];
-    for (int i = 0; i < nProjExprs; i++) {
-      RexNode newExpr = mergedProgram.expandLocalRef(projList.get(i));
-      if (joinType != JoinRelType.INNER) {
-        newExpr =
-            newExpr.accept(
-                new RelOptUtil.RexInputConverter(
-                    rexBuilder,
-                    joinChildrenRowType.getFieldList(),
-                    newJoinFields,
-                    adjustments));
-      }
-      newProjExprs.add(newExpr);
-    }
+  public static final HiveJoinProjectTransposeRule RIGHT_PROJECT_INCLUDE_OUTER =
+      new HiveJoinProjectTransposeRule(
+          operand(
+              HiveJoin.class,
+              operand(RelNode.class, any()),
+              operand(HiveProject.class, any())),
+          "Join(IncludingOuter)ProjectTransposeRule(Other-Project)",
+          true, HiveProject.DEFAULT_PROJECT_FACTORY);
 
-    // finally, create the projection on top of the join
-    RelNode newProjRel = projectFactory.createProject(newJoinRel, newProjExprs,
-        joinRel.getRowType().getFieldNames());
 
-    call.transformTo(newProjRel);
+  private HiveJoinProjectTransposeRule(
+      RelOptRuleOperand operand, String description,
+      boolean includeOuter, ProjectFactory projectFactory) {
+    super(operand, description, includeOuter, projectFactory);
   }
 
-  /**
-   * Creates projection expressions corresponding to one of the inputs into
-   * the join
-   *
-   * @param projRel            the projection input into the join (if it exists)
-   * @param joinChild          the child of the projection input (if there is a
-   *                           projection); otherwise, this is the join input
-   * @param adjustmentAmount   the amount the expressions need to be shifted by
-   * @param rexBuilder         rex builder
-   * @param joinChildrenFields concatenation of the fields from the left and
-   *                           right join inputs (once the projections have been
-   *                           removed)
-   * @param projects           Projection expressions &amp; names to be created
-   */
-  private void createProjectExprs(
-      Project projRel,
-      RelNode joinChild,
-      int adjustmentAmount,
-      RexBuilder rexBuilder,
-      List<RelDataTypeField> joinChildrenFields,
-      List<Pair<RexNode, String>> projects) {
-    List<RelDataTypeField> childFields =
-        joinChild.getRowType().getFieldList();
-    if (projRel != null) {
-      List<Pair<RexNode, String>> namedProjects =
-          projRel.getNamedProjects();
-      int nChildFields = childFields.size();
-      int[] adjustments = new int[nChildFields];
-      for (int i = 0; i < nChildFields; i++) {
-        adjustments[i] = adjustmentAmount;
-      }
-      for (Pair<RexNode, String> pair : namedProjects) {
-        RexNode e = pair.left;
-        if (adjustmentAmount != 0) {
-          // shift the references by the adjustment amount
-          e = e.accept(
-              new RelOptUtil.RexInputConverter(
-                  rexBuilder,
-                  childFields,
-                  joinChildrenFields,
-                  adjustments));
-        }
-        projects.add(Pair.of(e, pair.right));
-      }
-    } else {
-      // no projection; just create references to the inputs
-      for (int i = 0; i < childFields.size(); i++) {
-        final RelDataTypeField field = childFields.get(i);
-        projects.add(
-            Pair.of(
-                (RexNode) rexBuilder.makeInputRef(
-                    field.getType(),
-                    i + adjustmentAmount),
-                field.getName()));
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/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 f26d1df..73ae7c4 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
@@ -914,8 +914,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
       if (HiveConf.getBoolVar(conf, ConfVars.HIVE_CBO_RETPATH_HIVEOP)) {
         // 6.1. Merge join into multijoin operators (if possible)
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, true, mdProvider.getMetadataProvider(),
-                HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.BOTH_PROJECT,
-                HiveJoinProjectTransposeRule.LEFT_PROJECT, HiveJoinProjectTransposeRule.RIGHT_PROJECT,
+                HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.BOTH_PROJECT_INCLUDE_OUTER,
+                HiveJoinProjectTransposeRule.LEFT_PROJECT_INCLUDE_OUTER,
+                HiveJoinProjectTransposeRule.RIGHT_PROJECT_INCLUDE_OUTER,
                 HiveJoinToMultiJoinRule.INSTANCE, HiveProjectMergeRule.INSTANCE);
         // The previous rules can pull up projections through join operators,
         // thus we run the field trimmer again to push them back down

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/constprog_partitioner.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/constprog_partitioner.q.out b/ql/src/test/results/clientpositive/constprog_partitioner.q.out
index 6475fa7..e22a4a9 100644
--- a/ql/src/test/results/clientpositive/constprog_partitioner.q.out
+++ b/ql/src/test/results/clientpositive/constprog_partitioner.q.out
@@ -111,42 +111,42 @@ STAGE PLANS:
               predicate: ((l_linenumber = 1) and l_orderkey is not null) (type: boolean)
               Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3
+                expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
-                  key expressions: _col0 (type: int), _col3 (type: int)
-                  sort order: ++
-                  Map-reduce partition columns: _col0 (type: int), _col3 (type: int)
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: int), _col2 (type: int)
           TableScan
             alias: li
             Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (((l_shipmode = 'AIR') and l_orderkey is not null) and l_linenumber is not null) (type: boolean)
+              predicate: (((l_shipmode = 'AIR') and (l_linenumber = 1)) and l_orderkey is not null) (type: boolean)
               Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: l_orderkey (type: int), l_linenumber (type: int)
-                outputColumnNames: _col0, _col1
+                expressions: l_orderkey (type: int)
+                outputColumnNames: _col0
                 Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
-                  keys: _col0 (type: int), _col1 (type: int)
+                  keys: _col0 (type: int)
                   mode: hash
-                  outputColumnNames: _col0, _col1
+                  outputColumnNames: _col0
                   Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
-                    key expressions: _col0 (type: int), _col1 (type: int)
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Join Operator
           condition map:
                Left Semi Join 0 to 1
           keys:
-            0 _col0 (type: int), _col3 (type: int)
-            1 _col0 (type: int), _col1 (type: int)
+            0 _col0 (type: int)
+            1 _col0 (type: int)
           outputColumnNames: _col1, _col2
           Statistics: Num rows: 27 Data size: 3298 Basic stats: COMPLETE Column stats: NONE
           Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/correlationoptimizer10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer10.q.out b/ql/src/test/results/clientpositive/correlationoptimizer10.q.out
index 14a6ade..90fb6f4 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer10.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer10.q.out
@@ -474,17 +474,17 @@ STAGE PLANS:
             alias: xx
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+              predicate: (((UDFToDouble(key) < 200.0) and (UDFToDouble(key) > 20.0)) and key is not null) (type: boolean)
+              Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: string)
           TableScan
             Reduce Output Operator
@@ -581,17 +581,17 @@ STAGE PLANS:
             alias: xx
             Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+              predicate: (((UDFToDouble(key) < 200.0) and (UDFToDouble(key) > 20.0)) and key is not null) (type: boolean)
+              Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: string)
           TableScan
             alias: x
@@ -625,9 +625,9 @@ STAGE PLANS:
                   Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Demux Operator
-          Statistics: Num rows: 69 Data size: 693 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 57 Data size: 601 Basic stats: COMPLETE Column stats: NONE
           Mux Operator
-            Statistics: Num rows: 69 Data size: 693 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 57 Data size: 601 Basic stats: COMPLETE Column stats: NONE
             Join Operator
               condition map:
                    Left Semi Join 0 to 1
@@ -652,7 +652,7 @@ STAGE PLANS:
             outputColumnNames: _col0
             Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
             Mux Operator
-              Statistics: Num rows: 69 Data size: 693 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 57 Data size: 601 Basic stats: COMPLETE Column stats: NONE
               Join Operator
                 condition map:
                      Left Semi Join 0 to 1
@@ -802,17 +802,17 @@ STAGE PLANS:
             alias: xx
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              predicate: (((UDFToDouble(key) < 200.0) and (UDFToDouble(key) > 180.0)) and key is not null) (type: boolean)
+              Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: string)
           TableScan
             Reduce Output Operator
@@ -828,10 +828,10 @@ STAGE PLANS:
             0 _col0 (type: string)
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 33 Data size: 358 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 33 Data size: 358 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -921,17 +921,17 @@ STAGE PLANS:
             alias: xx
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              predicate: (((UDFToDouble(key) < 200.0) and (UDFToDouble(key) > 180.0)) and key is not null) (type: boolean)
+              Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: string)
           TableScan
             alias: xx
@@ -965,9 +965,9 @@ STAGE PLANS:
                   Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Demux Operator
-          Statistics: Num rows: 306 Data size: 3250 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 84 Data size: 891 Basic stats: COMPLETE Column stats: NONE
           Mux Operator
-            Statistics: Num rows: 306 Data size: 3250 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 84 Data size: 891 Basic stats: COMPLETE Column stats: NONE
             Join Operator
               condition map:
                    Left Semi Join 0 to 1
@@ -992,7 +992,7 @@ STAGE PLANS:
             outputColumnNames: _col0
             Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
             Mux Operator
-              Statistics: Num rows: 306 Data size: 3250 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 84 Data size: 891 Basic stats: COMPLETE Column stats: NONE
               Join Operator
                 condition map:
                      Left Semi Join 0 to 1

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/spark/constprog_partitioner.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/constprog_partitioner.q.out b/ql/src/test/results/clientpositive/spark/constprog_partitioner.q.out
index 665bfce..3a3a751 100644
--- a/ql/src/test/results/clientpositive/spark/constprog_partitioner.q.out
+++ b/ql/src/test/results/clientpositive/spark/constprog_partitioner.q.out
@@ -124,13 +124,13 @@ STAGE PLANS:
                     predicate: ((l_linenumber = 1) and l_orderkey is not null) (type: boolean)
                     Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int)
-                      outputColumnNames: _col0, _col1, _col2, _col3
+                      expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int)
+                      outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int), _col3 (type: int)
-                        sort order: ++
-                        Map-reduce partition columns: _col0 (type: int), _col3 (type: int)
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int)
         Map 3 
@@ -139,21 +139,21 @@ STAGE PLANS:
                   alias: li
                   Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((l_shipmode = 'AIR') and l_orderkey is not null) and l_linenumber is not null) (type: boolean)
+                    predicate: (((l_shipmode = 'AIR') and (l_linenumber = 1)) and l_orderkey is not null) (type: boolean)
                     Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: l_orderkey (type: int), l_linenumber (type: int)
-                      outputColumnNames: _col0, _col1
+                      expressions: l_orderkey (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        keys: _col0 (type: int), _col1 (type: int)
+                        keys: _col0 (type: int)
                         mode: hash
-                        outputColumnNames: _col0, _col1
+                        outputColumnNames: _col0
                         Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
-                          key expressions: _col0 (type: int), _col1 (type: int)
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                          key expressions: _col0 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
                           Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
         Reducer 2 
             Reduce Operator Tree:
@@ -161,8 +161,8 @@ STAGE PLANS:
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 _col0 (type: int), _col3 (type: int)
-                  1 _col0 (type: int), _col1 (type: int)
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
                 outputColumnNames: _col1, _col2
                 Statistics: Num rows: 27 Data size: 3298 Basic stats: COMPLETE Column stats: NONE
                 Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/spark/subquery_exists.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/subquery_exists.q.out b/ql/src/test/results/clientpositive/spark/subquery_exists.q.out
index 28eda26..44f467b 100644
--- a/ql/src/test/results/clientpositive/spark/subquery_exists.q.out
+++ b/ql/src/test/results/clientpositive/spark/subquery_exists.q.out
@@ -41,17 +41,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (value is not null and key is not null) (type: boolean)
-                    Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((value > 'val_9') and key is not null) (type: boolean)
+                    Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col1 (type: string), _col0 (type: string)
                         sort order: ++
                         Map-reduce partition columns: _col1 (type: string), _col0 (type: string)
-                        Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
         Map 3 
             Map Operator Tree:
                 TableScan
@@ -83,10 +83,10 @@ STAGE PLANS:
                   0 _col1 (type: string), _col0 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/spark/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/subquery_in.q.out b/ql/src/test/results/clientpositive/spark/subquery_in.q.out
index 00b3399..bfcdaa8 100644
--- a/ql/src/test/results/clientpositive/spark/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/spark/subquery_in.q.out
@@ -31,17 +31,17 @@ STAGE PLANS:
                   alias: src
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: key is not null (type: boolean)
-                    Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                    predicate: (key > '9') (type: boolean)
+                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: string)
         Map 3 
             Map Operator Tree:
@@ -74,10 +74,10 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -149,17 +149,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (key is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((key > '9') and value is not null) (type: boolean)
+                    Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
         Map 3 
             Map Operator Tree:
                 TableScan
@@ -191,10 +191,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -627,17 +627,17 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (key is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                    predicate: ((key > '9') and value is not null) (type: boolean)
+                    Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                        Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
         Map 3 
             Map Operator Tree:
                 TableScan
@@ -669,10 +669,10 @@ STAGE PLANS:
                   0 _col0 (type: string), _col1 (type: string)
                   1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out b/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out
index a2dd910..be39d0d 100644
--- a/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out
+++ b/ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out
@@ -205,21 +205,21 @@ STAGE PLANS:
                   alias: lineitem
                   Statistics: Num rows: 100 Data size: 11999 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((l_shipmode = 'AIR') and l_orderkey is not null) and l_linenumber is not null) (type: boolean)
+                    predicate: (((l_shipmode = 'AIR') and (l_linenumber = 1)) and l_orderkey is not null) (type: boolean)
                     Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: l_orderkey (type: int), l_linenumber (type: int)
-                      outputColumnNames: _col0, _col1
+                      expressions: l_orderkey (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
-                        keys: _col0 (type: int), _col1 (type: int)
+                        keys: _col0 (type: int)
                         mode: hash
-                        outputColumnNames: _col0, _col1
+                        outputColumnNames: _col0
                         Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                         Spark HashTable Sink Operator
                           keys:
-                            0 _col0 (type: int), _col3 (type: int)
-                            1 _col0 (type: int), _col1 (type: int)
+                            0 _col0 (type: int)
+                            1 _col0 (type: int)
             Local Work:
               Map Reduce Local Work
 
@@ -236,15 +236,15 @@ STAGE PLANS:
                     predicate: (((l_linenumber = 1) and l_orderkey is not null) and l_partkey is not null) (type: boolean)
                     Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), 1 (type: int)
-                      outputColumnNames: _col0, _col1, _col2, _col3
+                      expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int)
+                      outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 13 Data size: 1559 Basic stats: COMPLETE Column stats: NONE
                       Map Join Operator
                         condition map:
                              Left Semi Join 0 to 1
                         keys:
-                          0 _col0 (type: int), _col3 (type: int)
-                          1 _col0 (type: int), _col1 (type: int)
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
                         outputColumnNames: _col1, _col2
                         input vertices:
                           1 Map 2

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/subquery_exists.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_exists.q.out b/ql/src/test/results/clientpositive/subquery_exists.q.out
index 776b145..ace5efe 100644
--- a/ql/src/test/results/clientpositive/subquery_exists.q.out
+++ b/ql/src/test/results/clientpositive/subquery_exists.q.out
@@ -36,17 +36,17 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (value is not null and key is not null) (type: boolean)
-              Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+              predicate: ((value > 'val_9') and key is not null) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col1 (type: string), _col0 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col1 (type: string), _col0 (type: string)
-                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
           TableScan
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -75,10 +75,10 @@ STAGE PLANS:
             0 _col1 (type: string), _col0 (type: string)
             1 _col0 (type: string), _col1 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_in.q.out b/ql/src/test/results/clientpositive/subquery_in.q.out
index 7c53638..f82c799 100644
--- a/ql/src/test/results/clientpositive/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/subquery_in.q.out
@@ -26,17 +26,17 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              predicate: (key > '9') (type: boolean)
+              Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: string)
           TableScan
             alias: src
@@ -66,10 +66,10 @@ STAGE PLANS:
             0 _col0 (type: string)
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -136,17 +136,17 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (key is not null and value is not null) (type: boolean)
-              Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+              predicate: ((key > '9') and value is not null) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
           TableScan
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -175,10 +175,10 @@ STAGE PLANS:
             0 _col0 (type: string), _col1 (type: string)
             1 _col0 (type: string), _col1 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -671,17 +671,17 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (key is not null and value is not null) (type: boolean)
-              Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+              predicate: ((key > '9') and value is not null) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
           TableScan
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: string)
@@ -696,10 +696,10 @@ STAGE PLANS:
             0 _col0 (type: string), _col1 (type: string)
             1 _col0 (type: string), _col1 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/subquery_in_having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_in_having.q.out b/ql/src/test/results/clientpositive/subquery_in_having.q.out
index 8cabfa7..da1da06 100644
--- a/ql/src/test/results/clientpositive/subquery_in_having.q.out
+++ b/ql/src/test/results/clientpositive/subquery_in_having.q.out
@@ -782,17 +782,17 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              predicate: (key > '8') (type: boolean)
+              Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col1 (type: string)
           TableScan
             alias: b
@@ -822,13 +822,13 @@ STAGE PLANS:
             0 _col0 (type: string)
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
           Group By Operator
             aggregations: count()
             keys: _col0 (type: string), _col1 (type: string)
             mode: hash
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -844,7 +844,7 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string)
               sort order: ++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-              Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col2 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -852,10 +852,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
           Filter Operator
             predicate: _col2 is not null (type: boolean)
-            Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -871,7 +871,7 @@ STAGE PLANS:
               key expressions: _col2 (type: bigint)
               sort order: +
               Map-reduce partition columns: _col2 (type: bigint)
-              Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col0 (type: string), _col1 (type: string)
           TableScan
             Reduce Output Operator
@@ -887,10 +887,10 @@ STAGE PLANS:
             0 _col2 (type: bigint)
             1 _col0 (type: bigint)
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1079,10 +1079,10 @@ STAGE PLANS:
                 0 _col2 (type: bigint)
                 1 _col0 (type: bigint)
               outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
-                Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1098,7 +1098,7 @@ STAGE PLANS:
               key expressions: _col2 (type: bigint)
               sort order: +
               Map-reduce partition columns: _col2 (type: bigint)
-              Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col0 (type: string), _col1 (type: string)
           TableScan
             Reduce Output Operator
@@ -1114,10 +1114,10 @@ STAGE PLANS:
             0 _col2 (type: bigint)
             1 _col0 (type: bigint)
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1158,12 +1158,12 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              predicate: (key > '8') (type: boolean)
+              Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                 Map Join Operator
                   condition map:
                        Left Semi Join 0 to 1
@@ -1171,18 +1171,18 @@ STAGE PLANS:
                     0 _col0 (type: string)
                     1 _col0 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: string), _col1 (type: string)
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                      Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col2 (type: bigint)
       Local Work:
         Map Reduce Local Work
@@ -1192,10 +1192,10 @@ STAGE PLANS:
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
           Filter Operator
             predicate: _col2 is not null (type: boolean)
-            Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out b/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
index 5d6d4a8..76d7503 100644
--- a/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
+++ b/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
@@ -52,7 +52,7 @@ STAGE PLANS:
             alias: src11
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             Filter Operator
-              predicate: (key1 is not null and value1 is not null) (type: boolean)
+              predicate: ((key1 > '9') and value1 is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
                 expressions: key1 (type: string), value1 (type: string)
@@ -122,17 +122,17 @@ STAGE PLANS:
             alias: a
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (key is not null and value is not null) (type: boolean)
-              Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+              predicate: ((key > '9') and value is not null) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
           TableScan
             alias: a
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -161,10 +161,10 @@ STAGE PLANS:
             0 _col0 (type: string), _col1 (type: string)
             1 _col0 (type: string), _col1 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -588,17 +588,17 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (key is not null and value is not null) (type: boolean)
-              Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+              predicate: ((key > '9') and value is not null) (type: boolean)
+              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
           TableScan
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: string)
@@ -613,10 +613,10 @@ STAGE PLANS:
             0 _col0 (type: string), _col1 (type: string)
             1 _col0 (type: string), _col1 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/c40382d4/ql/src/test/results/clientpositive/subquery_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_views.q.out b/ql/src/test/results/clientpositive/subquery_views.q.out
index 41834a3..c59d86e 100644
--- a/ql/src/test/results/clientpositive/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/subquery_views.q.out
@@ -70,7 +70,7 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@cv2
 Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[50][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[52][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
 PREHOOK: query: explain
 select * 
 from cv2 where cv2.key in (select key from cv2 c where c.key < '11')
@@ -97,7 +97,7 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: ((value > 'val_11') and (key is null or value is null)) (type: boolean)
+              predicate: ((value > 'val_11') and (key is null or value is null or key is null)) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
@@ -135,15 +135,15 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: key is not null (type: boolean)
-              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              predicate: (key < '11') (type: boolean)
+              Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   sort order: 
-                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: string), _col1 (type: string)
           TableScan
             Reduce Output Operator
@@ -157,7 +157,7 @@ STAGE PLANS:
             0 
             1 
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -173,22 +173,22 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col0 (type: string)
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col0 (type: string)
-              Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
           TableScan
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: ((value > 'val_11') and key is not null) (type: boolean)
-              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+              predicate: (((value > 'val_11') and (key < '11')) and key is not null) (type: boolean)
+              Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string), key (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                   sort order: +++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
-                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 28 Data size: 297 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -197,14 +197,14 @@ STAGE PLANS:
             0 _col0 (type: string), _col1 (type: string), _col0 (type: string)
             1 _col0 (type: string), _col1 (type: string), _col2 (type: string)
           outputColumnNames: _col0, _col1, _col3
-          Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 200 Data size: 2132 Basic stats: COMPLETE Column stats: NONE
           Filter Operator
             predicate: _col3 is null (type: boolean)
-            Statistics: Num rows: 151 Data size: 1606 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 100 Data size: 1066 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: _col0 (type: string), _col1 (type: string)
               outputColumnNames: _col0, _col1
-              Statistics: Num rows: 151 Data size: 1606 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 100 Data size: 1066 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
                 table:
@@ -220,7 +220,7 @@ STAGE PLANS:
               key expressions: _col0 (type: string)
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 151 Data size: 1606 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 100 Data size: 1066 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: string)
           TableScan
             Reduce Output Operator
@@ -236,10 +236,10 @@ STAGE PLANS:
             0 _col0 (type: string)
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1
-          Statistics: Num rows: 166 Data size: 1766 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 110 Data size: 1172 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 166 Data size: 1766 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 110 Data size: 1172 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -252,7 +252,7 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: ((value > 'val_11') and (key is null or value is null)) (type: boolean)
+              predicate: ((value > 'val_11') and (key is null or value is null or key is null)) (type: boolean)
               Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
@@ -379,7 +379,7 @@ STAGE PLANS:
         ListSink
 
 Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[50][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[52][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-6:MAPRED' is a cross product
 PREHOOK: query: select * 
 from cv2 where cv2.key in (select key from cv2 c where c.key < '11')
 PREHOOK: type: QUERY