You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by we...@apache.org on 2017/05/24 23:52:27 UTC

[53/54] [abbrv] hive git commit: HIVE-15160: Can't order by an unselected column (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

HIVE-15160: Can't order by an unselected column (Pengcheng Xiong, 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/a74c1e7c
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a74c1e7c
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a74c1e7c

Branch: refs/heads/hive-14535
Commit: a74c1e7ceeab05917fbe01e9a7cd3eda8e87ee27
Parents: bda5e1e
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Wed May 24 15:05:55 2017 -0700
Committer: Pengcheng Xiong <px...@hortonworks.com>
Committed: Wed May 24 15:05:55 2017 -0700

----------------------------------------------------------------------
 .../rules/HiveProjectSortTransposeRule.java     |  56 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    | 198 ++++--
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   3 +
 .../hive/ql/parse/TypeCheckProcFactory.java     |   2 +-
 .../queries/clientpositive/order_by_expr_1.q    |  44 ++
 .../queries/clientpositive/order_by_expr_2.q    |  11 +
 .../clientpositive/annotate_stats_select.q.out  |  25 +-
 ql/src/test/results/clientpositive/cp_sel.q.out |  22 +-
 .../results/clientpositive/druid_basic2.q.out   |  10 +-
 .../clientpositive/dynamic_rdd_cache.q.out      |   6 +-
 .../groupby_grouping_sets_grouping.q.out        |   6 +-
 .../clientpositive/llap/bucket_groupby.q.out    |   2 +-
 .../clientpositive/llap/explainuser_1.q.out     | 221 +++---
 .../clientpositive/llap/limit_pushdown.q.out    |   6 +-
 .../clientpositive/llap/limit_pushdown3.q.out   |   6 +-
 .../llap/offset_limit_ppd_optimizer.q.out       |   6 +-
 .../clientpositive/llap/subquery_in.q.out       |  10 +-
 .../clientpositive/llap/vector_coalesce.q.out   |  96 +--
 .../clientpositive/llap/vector_date_1.q.out     |  30 +-
 .../clientpositive/llap/vector_decimal_2.q.out  | 297 +++-----
 .../llap/vector_decimal_round.q.out             |  26 +-
 .../vector_groupby_grouping_sets_grouping.q.out |   6 +-
 .../vector_groupby_grouping_sets_limit.q.out    |   2 +-
 .../clientpositive/llap/vector_interval_1.q.out |  12 +-
 .../llap/vector_interval_arithmetic.q.out       |  67 +-
 ql/src/test/results/clientpositive/order3.q.out |  22 +-
 .../clientpositive/order_by_expr_1.q.out        | 566 +++++++++++++++
 .../clientpositive/order_by_expr_2.q.out        | 100 +++
 ql/src/test/results/clientpositive/pcr.q.out    | 114 ++--
 .../results/clientpositive/perf/query31.q.out   |  68 +-
 .../results/clientpositive/perf/query36.q.out   |  20 +-
 .../results/clientpositive/perf/query39.q.out   |  34 +-
 .../results/clientpositive/perf/query42.q.out   |  30 +-
 .../results/clientpositive/perf/query52.q.out   |  30 +-
 .../results/clientpositive/perf/query64.q.out   | 154 ++---
 .../results/clientpositive/perf/query66.q.out   |  56 +-
 .../results/clientpositive/perf/query70.q.out   |  30 +-
 .../results/clientpositive/perf/query75.q.out   | 682 ++++++++++---------
 .../results/clientpositive/perf/query81.q.out   | 274 ++++----
 .../results/clientpositive/perf/query85.q.out   |  34 +-
 .../results/clientpositive/perf/query86.q.out   |  16 +-
 .../results/clientpositive/perf/query89.q.out   |  22 +-
 .../results/clientpositive/perf/query91.q.out   |  28 +-
 .../results/clientpositive/pointlookup2.q.out   |  94 +--
 .../results/clientpositive/pointlookup3.q.out   |  96 +--
 .../results/clientpositive/ppd_udf_case.q.out   |  36 +-
 .../spark/dynamic_rdd_cache.q.out               |   6 +-
 .../clientpositive/spark/limit_pushdown.q.out   |   2 +-
 .../test/results/clientpositive/spark/pcr.q.out |  44 +-
 .../clientpositive/spark/subquery_in.q.out      |   6 +-
 .../clientpositive/vector_coalesce.q.out        |  38 +-
 .../results/clientpositive/vector_date_1.q.out  |  25 +-
 .../clientpositive/vector_decimal_round.q.out   |  24 +-
 .../clientpositive/vector_interval_1.q.out      |  12 +-
 .../vector_interval_arithmetic.q.out            |  50 +-
 .../results/clientpositive/view_alias.q.out     |  40 +-
 56 files changed, 2269 insertions(+), 1654 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
index 1487ed4..871c411 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
@@ -18,22 +18,33 @@
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
 import org.apache.calcite.plan.RelOptCluster;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexCallBinding;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.util.mapping.Mappings;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveRelNode;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 
 import com.google.common.collect.ImmutableList;
@@ -67,15 +78,15 @@ public class HiveProjectSortTransposeRule extends RelOptRule {
     final HiveSortLimit sort = call.rel(1);
     final RelOptCluster cluster = project.getCluster();
 
-    // Determine mapping between project input and output fields. If sort
-    // relies on non-trivial expressions, we can't push.
+    // Determine mapping between project input and output fields. 
+    // In Hive, Sort is always based on RexInputRef
+    // We only need to check if project can contain all the positions that sort needs.
     final Mappings.TargetMapping map =
         RelOptUtil.permutationIgnoreCast(
             project.getProjects(), project.getInput().getRowType()).inverse();
+    Set<Integer> needed = new HashSet<>();
     for (RelFieldCollation fc : sort.getCollation().getFieldCollations()) {
-      if (map.getTarget(fc.getFieldIndex()) < 0) {
-        return;
-      }
+      needed.add(fc.getFieldIndex());
       final RexNode node = project.getProjects().get(map.getTarget(fc.getFieldIndex()));
       if (node.isA(SqlKind.CAST)) {
         // Check whether it is a monotonic preserving cast, otherwise we cannot push
@@ -88,12 +99,35 @@ public class HiveProjectSortTransposeRule extends RelOptRule {
         }
       }
     }
+    Map<Integer,Integer> m = new HashMap<>();
+    for (int projPos = 0; projPos < project.getChildExps().size(); projPos++) {
+      RexNode expr = project.getChildExps().get(projPos);
+      if (expr instanceof RexInputRef) {
+        Set<Integer> positions = HiveCalciteUtil.getInputRefs(expr);
+        if (positions.size() > 1) {
+          continue;
+        } else {
+          int parentPos = positions.iterator().next();
+          if(needed.contains(parentPos)){
+            m.put(parentPos, projPos);
+            needed.remove(parentPos);
+          }
+        }
+      }
+    }
+    if(!needed.isEmpty()){
+      return;
+    }
+    
+    List<RelFieldCollation> fieldCollations = new ArrayList<>();
+    for (RelFieldCollation fc : sort.getCollation().getFieldCollations()) {
+      fieldCollations.add(new RelFieldCollation(m.get(fc.getFieldIndex()), fc.direction,
+          fc.nullDirection));
+    }
 
-    // Create new collation
-    final RelCollation newCollation =
-        RelCollationTraitDef.INSTANCE.canonize(
-            RexUtil.apply(map, sort.getCollation()));
-
+    RelTraitSet traitSet = sort.getCluster().traitSetOf(HiveRelNode.CONVENTION);
+    RelCollation newCollation = traitSet.canonize(RelCollationImpl.of(fieldCollations));
+    
     // New operators
     final RelNode newProject = project.copy(sort.getInput().getTraitSet(),
             ImmutableList.<RelNode>of(sort.getInput()));

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/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 fa96e94..721dac8 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
@@ -1744,9 +1744,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // 8. Merge, remove and reduce Project if possible
       perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
       basePlan = hepPlan(basePlan, false, mdProvider, executorProvider,
-           HiveProjectMergeRule.INSTANCE, ProjectRemoveRule.INSTANCE);
+           HiveProjectMergeRule.INSTANCE, ProjectRemoveRule.INSTANCE, HiveSortMergeRule.INSTANCE);
       perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER,
-        "Calcite: Prejoin ordering transformation, Merge Project-Project");
+          "Calcite: Prejoin ordering transformation, Merge Project-Project, Merge Sort-Sort");
 
       // 9. Rerun PPD through Project as column pruning would have introduced
       // DT above scans; By pushing filter just above TS, Hive can push it into
@@ -2887,7 +2887,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
           && selExprList.getChildCount() == 1 && selExprList.getChild(0).getChildCount() == 1) {
         ASTNode node = (ASTNode) selExprList.getChild(0).getChild(0);
         if (node.getToken().getType() == HiveParser.TOK_ALLCOLREF) {
-          srcRel = genSelectLogicalPlan(qb, srcRel, srcRel, null,null);
+          // As we said before, here we use genSelectLogicalPlan to rewrite AllColRef
+          srcRel = genSelectLogicalPlan(qb, srcRel, srcRel, null, null, true).getKey();
           RowResolver rr = this.relToHiveRR.get(srcRel);
           qbp.setSelExprForClause(detsClauseName, SemanticAnalyzer.genSelectDIAST(rr));
         }
@@ -3049,14 +3050,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
      * @param qb
      * @param srcRel
      * @param outermostOB
-     * @return Pair<RelNode, RelNode> Key- OB RelNode, Value - Input Select for
-     *         top constraining Select
+     * @return RelNode OB RelNode
      * @throws SemanticException
      */
-    private Pair<RelNode, RelNode> genOBLogicalPlan(QB qb, RelNode srcRel, boolean outermostOB)
-        throws SemanticException {
+    private RelNode genOBLogicalPlan(QB qb, Pair<RelNode, RowResolver> selPair,
+        boolean outermostOB) throws SemanticException {
+      // selPair.getKey() is the operator right before OB
+      // selPair.getValue() is RR which only contains columns needed in result
+      // set. Extra columns needed by order by will be absent from it.
+      RelNode srcRel = selPair.getKey();
+      RowResolver selectOutputRR = selPair.getValue();
       RelNode sortRel = null;
-      RelNode originalOBChild = null;
+      RelNode returnRel = null;
 
       QBParseInfo qbp = getQBParseInfo(qb);
       String dest = qbp.getClauseNames().iterator().next();
@@ -3064,7 +3069,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
       if (obAST != null) {
         // 1. OB Expr sanity test
-        // in strict mode, in the presence of order by, limit must be specified
+        // in strict mode, in the presence of order by, limit must be
+        // specified
         Integer limit = qb.getParseInfo().getDestLimit(dest);
         if (limit == null) {
           String error = StrictChecks.checkNoLimit(conf);
@@ -3096,11 +3102,28 @@ public class CalcitePlanner extends SemanticAnalyzer {
           obASTExpr = (ASTNode) obASTExprLst.get(i);
           nullObASTExpr = (ASTNode) obASTExpr.getChild(0);
           ASTNode ref = (ASTNode) nullObASTExpr.getChild(0);
-          Map<ASTNode, ExprNodeDesc> astToExprNDescMap = genAllExprNodeDesc(ref, inputRR);
-          ExprNodeDesc obExprNDesc = astToExprNDescMap.get(ref);
-          if (obExprNDesc == null)
+          Map<ASTNode, ExprNodeDesc> astToExprNDescMap = null;
+          ExprNodeDesc obExprNDesc = null;
+          // first try to get it from select
+          // in case of udtf, selectOutputRR may be null.
+          if (selectOutputRR != null) {
+            try {
+              astToExprNDescMap = genAllExprNodeDesc(ref, selectOutputRR);
+              obExprNDesc = astToExprNDescMap.get(ref);
+            } catch (SemanticException ex) {
+              // we can tolerate this as this is the previous behavior
+              LOG.debug("Can not find column in " + ref.getText() + ". The error msg is "
+                  + ex.getMessage());
+            }
+          }
+          // then try to get it from all
+          if (obExprNDesc == null) {
+            astToExprNDescMap = genAllExprNodeDesc(ref, inputRR);
+            obExprNDesc = astToExprNDescMap.get(ref);
+          }
+          if (obExprNDesc == null) {
             throw new SemanticException("Invalid order by expression: " + obASTExpr.toString());
-
+          }
           // 2.2 Convert ExprNode to RexNode
           rnd = converter.convert(obExprNDesc);
 
@@ -3126,8 +3149,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
           } else if (nullObASTExpr.getType() == HiveParser.TOK_NULLS_LAST) {
             nullOrder = RelFieldCollation.NullDirection.LAST;
           } else {
-            throw new SemanticException(
-                    "Unexpected null ordering option: " + nullObASTExpr.getType());
+            throw new SemanticException("Unexpected null ordering option: "
+                + nullObASTExpr.getType());
           }
 
           // 2.5 Add to field collations
@@ -3174,7 +3197,6 @@ public class CalcitePlanner extends SemanticAnalyzer {
                   "Duplicates detected when adding columns to RR: see previous message",
                   UnsupportedFeature.Duplicates_in_RR);
             }
-            originalOBChild = srcRel;
           }
         } else {
           if (!RowResolver.add(outputRR, inputRR)) {
@@ -3199,9 +3221,26 @@ public class CalcitePlanner extends SemanticAnalyzer {
             outputRR, sortRel);
         relToHiveRR.put(sortRel, outputRR);
         relToHiveColNameCalcitePosMap.put(sortRel, hiveColNameCalcitePosMap);
-      }
 
-      return (new Pair<RelNode, RelNode>(sortRel, originalOBChild));
+        if (selectOutputRR != null) {
+          List<RexNode> originalInputRefs = Lists.transform(srcRel.getRowType().getFieldList(),
+              new Function<RelDataTypeField, RexNode>() {
+                @Override
+                public RexNode apply(RelDataTypeField input) {
+                  return new RexInputRef(input.getIndex(), input.getType());
+                }
+              });
+          List<RexNode> selectedRefs = Lists.newArrayList();
+          for (int index = 0; index < selectOutputRR.getColumnInfos().size(); index++) {
+            selectedRefs.add(originalInputRefs.get(index));
+          }
+          // We need to add select since order by schema may have more columns than result schema.
+          returnRel = genSelectRelNode(selectedRefs, selectOutputRR, sortRel);
+        } else {
+          returnRel = sortRel;
+        }
+      }
+      return returnRel;
     }
 
     private RelNode genLimitLogicalPlan(QB qb, RelNode srcRel) throws SemanticException {
@@ -3532,8 +3571,20 @@ public class CalcitePlanner extends SemanticAnalyzer {
      *
      * @throws SemanticException
      */
-    private RelNode genSelectLogicalPlan(QB qb, RelNode srcRel, RelNode starSrcRel,
-                                         ImmutableMap<String, Integer> outerNameToPosMap, RowResolver outerRR)
+    /**
+     * @param qb
+     * @param srcRel
+     * @param starSrcRel
+     * @param outerNameToPosMap
+     * @param outerRR
+     * @param isAllColRefRewrite
+     *          when it is true, it means that it is called from group by *, where we use
+     *          genSelectLogicalPlan to rewrite *
+     * @return RelNode: the select relnode RowResolver: i.e., originalRR, the RR after select when there is an order by.
+     * @throws SemanticException
+     */
+    private Pair<RelNode,RowResolver> genSelectLogicalPlan(QB qb, RelNode srcRel, RelNode starSrcRel,
+                                         ImmutableMap<String, Integer> outerNameToPosMap, RowResolver outerRR, boolean isAllColRefRewrite)
         throws SemanticException {
       // 0. Generate a Select Node for Windowing
       // Exclude the newly-generated select columns from */etc. resolution.
@@ -3798,15 +3849,64 @@ public class CalcitePlanner extends SemanticAnalyzer {
       // 8. Build Calcite Rel
       RelNode outputRel = null;
       if (genericUDTF != null) {
-        // The basic idea for CBO support of UDTF is to treat UDTF as a special project.
-        // In AST return path, as we just need to generate a SEL_EXPR, we just need to remember the expressions and the alias.
-        // In OP return path, we need to generate a SEL and then a UDTF following old semantic analyzer.
-        outputRel = genUDTFPlan(genericUDTF, genericUDTFName, udtfTableAlias, udtfColAliases, qb, calciteColLst, out_rwsch, srcRel);
-      }
-      else{
-        outputRel = genSelectRelNode(calciteColLst, out_rwsch, srcRel);
+        // The basic idea for CBO support of UDTF is to treat UDTF as a special
+        // project.
+        // In AST return path, as we just need to generate a SEL_EXPR, we just
+        // need to remember the expressions and the alias.
+        // In OP return path, we need to generate a SEL and then a UDTF
+        // following old semantic analyzer.
+        outputRel = genUDTFPlan(genericUDTF, genericUDTFName, udtfTableAlias, udtfColAliases, qb,
+            calciteColLst, out_rwsch, srcRel);
+      } else {
+        String dest = qbp.getClauseNames().iterator().next();
+        ASTNode obAST = qbp.getOrderByForClause(dest);
+
+        RowResolver originalRR = null;
+        // We only support limited unselected column following by order by.
+        // TODO: support unselected columns in genericUDTF and windowing functions.
+        // We examine the order by in this query block and adds in column needed
+        // by order by in select list.
+        if (obAST != null && !(selForWindow != null && selExprList.getToken().getType() == HiveParser.TOK_SELECTDI) && !isAllColRefRewrite) {
+          // 1. OB Expr sanity test
+          // in strict mode, in the presence of order by, limit must be
+          // specified
+          Integer limit = qb.getParseInfo().getDestLimit(dest);
+          if (limit == null) {
+            String error = StrictChecks.checkNoLimit(conf);
+            if (error != null) {
+              throw new SemanticException(SemanticAnalyzer.generateErrorMessage(obAST, error));
+            }
+          }
+          List<RexNode> originalInputRefs = Lists.transform(srcRel.getRowType().getFieldList(),
+              new Function<RelDataTypeField, RexNode>() {
+                @Override
+                public RexNode apply(RelDataTypeField input) {
+                  return new RexInputRef(input.getIndex(), input.getType());
+                }
+              });
+          originalRR = out_rwsch.duplicate();
+          for (int i = 0; i < inputRR.getColumnInfos().size(); i++) {
+            ColumnInfo colInfo = new ColumnInfo(inputRR.getColumnInfos().get(i));
+            String internalName = SemanticAnalyzer.getColumnInternalName(out_rwsch.getColumnInfos()
+                .size() + i);
+            colInfo.setInternalName(internalName);
+            // if there is any confict, then we do not generate it in the new select
+            // otherwise, we add it into the calciteColLst and generate the new select
+            if (!out_rwsch.putWithCheck(colInfo.getTabAlias(), colInfo.getAlias(), internalName,
+                colInfo)) {
+              LOG.trace("Column already present in RR. skipping.");
+            } else {
+              calciteColLst.add(originalInputRefs.get(i));
+            }
+          }
+          outputRel = genSelectRelNode(calciteColLst, out_rwsch, srcRel);
+          // outputRel is the generated augmented select with extra unselected
+          // columns, and originalRR is the original generated select
+          return new Pair<RelNode, RowResolver>(outputRel, originalRR);
+        } else {
+          outputRel = genSelectRelNode(calciteColLst, out_rwsch, srcRel);
+        }
       }
-
       // 9. Handle select distinct as GBY if there exist windowing functions
       if (selForWindow != null && selExprList.getToken().getType() == HiveParser.TOK_SELECTDI) {
         ImmutableBitSet groupSet = ImmutableBitSet.range(outputRel.getRowType().getFieldList().size());
@@ -3824,7 +3924,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         this.relToHiveRR.put(outputRel, groupByOutputRowResolver);
       }
 
-      return outputRel;
+      return new Pair<RelNode, RowResolver>(outputRel, null);
     }
 
     private RelNode genUDTFPlan(GenericUDTF genericUDTF, String genericUDTFName, String outputTableAlias,
@@ -4051,51 +4151,19 @@ public class CalcitePlanner extends SemanticAnalyzer {
       srcRel = (gbHavingRel == null) ? srcRel : gbHavingRel;
 
       // 5. Build Rel for Select Clause
-      selectRel = genSelectLogicalPlan(qb, srcRel, starSrcRel, outerNameToPosMap, outerRR);
+      Pair<RelNode, RowResolver> selPair = genSelectLogicalPlan(qb, srcRel, starSrcRel, outerNameToPosMap, outerRR, false);
+      selectRel = selPair.getKey();
       srcRel = (selectRel == null) ? srcRel : selectRel;
 
       // 6. Build Rel for OB Clause
-      Pair<RelNode, RelNode> obTopProjPair = genOBLogicalPlan(qb, srcRel, outerMostQB);
-      obRel = obTopProjPair.getKey();
-      RelNode topConstrainingProjArgsRel = obTopProjPair.getValue();
+      obRel = genOBLogicalPlan(qb, selPair, outerMostQB);
       srcRel = (obRel == null) ? srcRel : obRel;
 
       // 7. Build Rel for Limit Clause
       limitRel = genLimitLogicalPlan(qb, srcRel);
       srcRel = (limitRel == null) ? srcRel : limitRel;
 
-      // 8. Introduce top constraining select if needed.
-      // NOTES:
-      // 1. Calcite can not take an expr in OB; hence it needs to be added as VC
-      // in the input select; In such cases we need to introduce a select on top
-      // to ensure VC is not visible beyond Limit, OB.
-      // 2. Hive can not preserve order across select. In subqueries OB is used
-      // to get a deterministic set of tuples from following limit. Hence we
-      // introduce the constraining select above Limit (if present) instead of
-      // OB.
-      // 3. The top level OB will not introduce constraining select due to Hive
-      // limitation(#2) stated above. The RR for OB will not include VC. Thus
-      // Result Schema will not include exprs used by top OB. During AST Conv,
-      // in the PlanModifierForASTConv we would modify the top level OB to
-      // migrate exprs from input sel to SortRel (Note that Calcite doesn't
-      // support this; but since we are done with Calcite at this point its OK).
-      if (topConstrainingProjArgsRel != null) {
-        List<RexNode> originalInputRefs = Lists.transform(topConstrainingProjArgsRel.getRowType()
-            .getFieldList(), new Function<RelDataTypeField, RexNode>() {
-          @Override
-          public RexNode apply(RelDataTypeField input) {
-            return new RexInputRef(input.getIndex(), input.getType());
-          }
-        });
-        RowResolver topConstrainingProjRR = new RowResolver();
-        if (!RowResolver.add(topConstrainingProjRR,
-            this.relToHiveRR.get(topConstrainingProjArgsRel))) {
-          LOG.warn("Duplicates detected when adding columns to RR: see previous message");
-        }
-        srcRel = genSelectRelNode(originalInputRefs, topConstrainingProjRR, srcRel);
-      }
-
-      // 9. Incase this QB corresponds to subquery then modify its RR to point
+      // 8. Incase this QB corresponds to subquery then modify its RR to point
       // to subquery alias
       // TODO: cleanup this
       if (qb.getParseInfo().getAlias() != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index d68f832..43a61ce 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -1617,6 +1617,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           throw new SemanticException(generateErrorMessage(ast,
               ErrorMsg.CLUSTERBY_ORDERBY_CONFLICT.getMsg()));
         }
+        // If there are aggregations in order by, we need to remember them in qb.
+        qbp.addAggregationExprsForClause(ctx_1.dest,
+            doPhase1GetAggregationsFromSelect(ast, qb, ctx_1.dest));
         break;
 
       case HiveParser.TOK_GROUPBY:

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/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 f678d0b..632b9c6 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
@@ -163,7 +163,7 @@ public class TypeCheckProcFactory {
     if (colInfo != null) {
       desc = new ExprNodeColumnDesc(colInfo);
       ASTNode source = input.getExpressionSource(expr);
-      if (source != null) {
+      if (source != null && ctx.getUnparseTranslator() != null) {
         ctx.getUnparseTranslator().addCopyTranslation(expr, source);
       }
       return desc;

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/queries/clientpositive/order_by_expr_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/order_by_expr_1.q b/ql/src/test/queries/clientpositive/order_by_expr_1.q
new file mode 100644
index 0000000..1d99e6a
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/order_by_expr_1.q
@@ -0,0 +1,44 @@
+set hive.fetch.task.conversion=none;
+
+create table t(a int, b int);
+
+insert into t values (1,2),(1,2),(1,3),(2,4),(20,-100),(-1000,100),(4,5),(3,7),(8,9);
+
+select a, count(a) from t group by a order by count(a), a;
+
+explain
+select
+  interval '2-2' year to month + interval '3-3' year to month,
+  interval '2-2' year to month - interval '3-3' year to month
+from t 
+order by interval '2-2' year to month + interval '3-3' year to month
+limit 2;
+
+select a,b, count(*) from t group by a, b order by a+b;
+select a,b, count(*) from t group by a, b order by count(*), b desc; 
+select a,b,count(*),a+b from t group by a, b order by a+b;
+select a,b from t order by a+b;
+select a,b,a+b from t order by a+b;
+select a,b,a+b from t order by a+b desc;
+select cast(0.99999999999999999999 as decimal(20,19)) as c from t limit 1;
+select cast(0.99999999999999999999 as decimal(20,19)) as c from t order by c limit 1;
+select a from t order by b;
+select a from t order by 0-b;
+select b from t order by 0-b;
+select b from t order by a, 0-b;
+select b from t order by a+1, 0-b;
+select b from t order by 0-b, a+1;
+explain select b from t order by 0-b, a+1;
+select a,b from t order by 0-b;
+select a,b from t order by a, a+1, 0-b;
+select a,b from t order by 0-b, a+1;
+select a+1,b from t order by a, a+1, 0-b;
+select a+1 as c, b from t order by a, a+1, 0-b;
+select a, a+1 as c, b from t order by a, a+1, 0-b;
+select a, a+1 as c, b, 2*b from t order by a, a+1, 0-b;
+explain select a, a+1 as c, b, 2*b from t order by a, a+1, 0-b;
+select a, a+1 as c, b, 2*b from t order by a+1, 0-b;
+select a,b, count(*) as c from t group by a, b order by c, a+b desc; 
+
+select a, max(b) from t group by a order by count(b), a desc; 
+select a, max(b) from t group by a order by count(b), a; 

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/queries/clientpositive/order_by_expr_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/order_by_expr_2.q b/ql/src/test/queries/clientpositive/order_by_expr_2.q
new file mode 100644
index 0000000..043f8ed
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/order_by_expr_2.q
@@ -0,0 +1,11 @@
+set hive.fetch.task.conversion=none;
+
+create table t(a int, b int);
+
+insert into t values (1,2),(1,2),(1,3),(2,4),(20,-100),(-1000,100),(4,5),(3,7),(8,9);
+
+select a as b, b as a from t order by a;
+select a as b, b as a from t order by t.a;
+select a as b from t order by b;
+select a as b from t order by 0-a;
+select a,b,count(*),a+b from t group by a, b order by a+b;

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/annotate_stats_select.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_select.q.out b/ql/src/test/results/clientpositive/annotate_stats_select.q.out
index 873f1ab..67d134b 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_select.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_select.q.out
@@ -883,8 +883,7 @@ POSTHOOK: query: explain select h, 11.0 from (select hell as h from (select i1,
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
   Stage: Stage-1
@@ -906,28 +905,6 @@ STAGE PLANS:
         Limit
           Number of rows: 10
           Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-            File Output Operator
-              compressed: false
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              sort order: 
-              Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-              TopN Hash Memory Usage: 0.1
-      Reduce Operator Tree:
-        Limit
-          Number of rows: 10
-          Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: 'hello' (type: string), 11 (type: int)
             outputColumnNames: _col0, _col1

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/cp_sel.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cp_sel.q.out b/ql/src/test/results/clientpositive/cp_sel.q.out
index 1778ccd..af2efeb 100644
--- a/ql/src/test/results/clientpositive/cp_sel.q.out
+++ b/ql/src/test/results/clientpositive/cp_sel.q.out
@@ -27,23 +27,27 @@ STAGE PLANS:
                 value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), 'hello' (type: string), 'world' (type: string)
-          outputColumnNames: _col0, _col1, _col2, _col3
+          expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
+          outputColumnNames: _col0, _col1
           Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
           Limit
             Number of rows: 1
             Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
+            Select Operator
+              expressions: _col0 (type: string), _col1 (type: string), 'hello' (type: string), 'world' (type: string)
+              outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              File Output Operator
+                compressed: false
+                Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
-      limit: 1
+      limit: -1
       Processor Tree:
         ListSink
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/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 6177d56..a3f05be 100644
--- a/ql/src/test/results/clientpositive/druid_basic2.q.out
+++ b/ql/src/test/results/clientpositive/druid_basic2.q.out
@@ -559,11 +559,11 @@ STAGE PLANS:
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             GatherStats: false
             Select Operator
-              expressions: robot (type: string), __time (type: timestamp), $f3 (type: float), $f4 (type: float)
-              outputColumnNames: _col0, _col1, _col2, _col3
+              expressions: robot (type: string), __time (type: timestamp), $f3 (type: float), $f4 (type: float), UDFToInteger(robot) (type: int)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col5
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Reduce Output Operator
-                key expressions: UDFToInteger(_col0) (type: int), _col2 (type: float)
+                key expressions: _col5 (type: int), _col2 (type: float)
                 null sort order: az
                 sort order: +-
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -632,7 +632,7 @@ STAGE PLANS:
               name: default.druid_table_1
             name: default.druid_table_1
       Truncated Path -> Alias:
-        /druid_table_1 [druid_table_1]
+        /druid_table_1 [$hdt$_0:druid_table_1]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
@@ -667,7 +667,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 10
+      limit: -1
       Processor Tree:
         ListSink
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out b/ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out
index fc00309..1a0e46c 100644
--- a/ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out
+++ b/ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out
@@ -1109,7 +1109,7 @@ STAGE PLANS:
           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
           Select Operator
             expressions: _col1 (type: int), _col2 (type: int), _col3 (type: double), _col4 (type: double), _col6 (type: int), _col7 (type: int), _col8 (type: double), _col9 (type: double)
-            outputColumnNames: _col0, _col1, _col3, _col4, _col5, _col6, _col8, _col9
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
             Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
             File Output Operator
               compressed: true
@@ -1123,10 +1123,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col0 (type: int), _col1 (type: int), _col3 (type: double), _col4 (type: double), _col8 (type: double), _col9 (type: double)
+              key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: double), _col3 (type: double), _col6 (type: double), _col7 (type: double)
               sort order: ++++++
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              value expressions: _col5 (type: int), _col6 (type: int)
+              value expressions: _col4 (type: int), _col5 (type: int)
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), 3 (type: int), KEY.reducesinkkey2 (type: double), KEY.reducesinkkey3 (type: double), VALUE._col0 (type: int), VALUE._col1 (type: int), 4 (type: int), KEY.reducesinkkey4 (type: double), KEY.reducesinkkey5 (type: double)

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out b/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out
index 473d17a..39dd731 100644
--- a/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out
+++ b/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out
@@ -317,8 +317,8 @@ STAGE PLANS:
             predicate: ((grouping(_col2, 1) = 1) or (grouping(_col2, 0) = 1)) (type: boolean)
             Statistics: Num rows: 6 Data size: 60 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: _col0 (type: int), _col1 (type: int), (grouping(_col2, 1) + grouping(_col2, 0)) (type: int)
-              outputColumnNames: _col0, _col1, _col2
+              expressions: _col0 (type: int), _col1 (type: int), (grouping(_col2, 1) + grouping(_col2, 0)) (type: int), CASE WHEN (((grouping(_col2, 1) + grouping(_col2, 0)) = 1)) THEN (_col0) ELSE (null) END (type: int)
+              outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 6 Data size: 60 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -332,7 +332,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col2 (type: int), CASE WHEN ((_col2 = 1)) THEN (_col0) ELSE (null) END (type: int)
+              key expressions: _col2 (type: int), _col3 (type: int)
               sort order: -+
               Statistics: Num rows: 6 Data size: 60 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col0 (type: int), _col1 (type: int)

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out b/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
index d724131..d68797f 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_groupby.q.out
@@ -1626,7 +1626,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 10
+      limit: -1
       Processor Tree:
         ListSink
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index f701cab..8b04bc9 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -358,13 +358,13 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 4 llap
-      File Output Operator [FS_38]
+      File Output Operator [FS_39]
         Select Operator [SEL_37] (rows=1 width=20)
           Output:["_col0","_col1","_col2"]
         <-Reducer 3 [SIMPLE_EDGE] llap
           SHUFFLE [RS_36]
-            Select Operator [SEL_35] (rows=1 width=20)
-              Output:["_col0","_col1","_col2"]
+            Select Operator [SEL_35] (rows=1 width=28)
+              Output:["_col0","_col1","_col2","_col3"]
               Group By Operator [GBY_34] (rows=1 width=20)
                 Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
               <-Reducer 2 [SIMPLE_EDGE] llap
@@ -376,14 +376,14 @@ Stage-0
                       Output:["_col1","_col4"]
                       Filter Operator [FIL_29] (rows=1 width=20)
                         predicate:(((_col3 + _col6) >= 0) and ((_col3 > 0) or (_col1 >= 0)))
-                        Merge Join Operator [MERGEJOIN_48] (rows=3 width=20)
+                        Merge Join Operator [MERGEJOIN_49] (rows=3 width=20)
                           Conds:RS_25._col0=RS_26._col0(Inner),RS_26._col0=RS_27._col0(Inner),Output:["_col1","_col3","_col4","_col6"]
                         <-Map 1 [SIMPLE_EDGE] llap
                           SHUFFLE [RS_25]
                             PartitionCols:_col0
                             Select Operator [SEL_2] (rows=18 width=84)
                               Output:["_col0","_col1"]
-                              Filter Operator [FIL_45] (rows=18 width=84)
+                              Filter Operator [FIL_46] (rows=18 width=84)
                                 predicate:key is not null
                                 TableScan [TS_0] (rows=20 width=84)
                                   default@cbo_t3,cbo_t3,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int"]
@@ -395,7 +395,7 @@ Stage-0
                             <-Reducer 9 [SIMPLE_EDGE] llap
                               SHUFFLE [RS_22]
                                 Select Operator [SEL_20] (rows=1 width=105)
-                                  Output:["_col0","_col1","_col2","_col3"]
+                                  Output:["_col0","_col1","_col2","_col5"]
                                   Group By Operator [GBY_19] (rows=1 width=101)
                                     Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                   <-Map 8 [SIMPLE_EDGE] llap
@@ -403,7 +403,7 @@ Stage-0
                                       PartitionCols:_col0, _col1, _col2
                                       Group By Operator [GBY_17] (rows=2 width=101)
                                         Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
-                                        Filter Operator [FIL_47] (rows=5 width=93)
+                                        Filter Operator [FIL_48] (rows=5 width=93)
                                           predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and key is not null)
                                           TableScan [TS_14] (rows=20 width=88)
                                             default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
@@ -423,7 +423,7 @@ Stage-0
                                       PartitionCols:_col0, _col1, _col2
                                       Group By Operator [GBY_6] (rows=2 width=101)
                                         Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
-                                        Filter Operator [FIL_46] (rows=5 width=93)
+                                        Filter Operator [FIL_47] (rows=5 width=93)
                                           predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and key is not null)
                                           TableScan [TS_3] (rows=20 width=88)
                                             default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
@@ -447,69 +447,71 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 4 llap
-      File Output Operator [FS_37]
+      File Output Operator [FS_38]
         Select Operator [SEL_36] (rows=1 width=20)
           Output:["_col0","_col1","_col2"]
         <-Reducer 3 [SIMPLE_EDGE] llap
           SHUFFLE [RS_35]
-            Group By Operator [GBY_33] (rows=1 width=20)
-              Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
-            <-Reducer 2 [SIMPLE_EDGE] llap
-              SHUFFLE [RS_32]
-                PartitionCols:_col0, _col1
-                Group By Operator [GBY_31] (rows=1 width=20)
-                  Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col1, _col4
-                  Select Operator [SEL_30] (rows=1 width=20)
-                    Output:["_col1","_col4"]
-                    Filter Operator [FIL_26] (rows=1 width=20)
-                      predicate:(((_col3 + _col6) >= 0) and ((UDFToLong(_col1) + _col4) >= 0) and ((_col1 >= 1) or (_col4 >= 1)) and ((_col3 > 0) or (_col1 >= 0)))
-                      Merge Join Operator [MERGEJOIN_47] (rows=3 width=20)
-                        Conds:RS_22._col0=RS_23._col0(Inner),RS_23._col0=RS_24._col0(Inner),Output:["_col1","_col3","_col4","_col6"]
-                      <-Map 1 [SIMPLE_EDGE] llap
-                        SHUFFLE [RS_22]
-                          PartitionCols:_col0
-                          Select Operator [SEL_2] (rows=18 width=84)
-                            Output:["_col0","_col1"]
-                            Filter Operator [FIL_44] (rows=18 width=84)
-                              predicate:((c_int > 0) and key is not null)
-                              TableScan [TS_0] (rows=20 width=84)
-                                default@cbo_t3,cbo_t3,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int"]
-                      <-Reducer 7 [SIMPLE_EDGE] llap
-                        SHUFFLE [RS_23]
-                          PartitionCols:_col0
-                          Select Operator [SEL_12] (rows=1 width=97)
-                            Output:["_col0","_col1","_col2"]
-                          <-Reducer 6 [SIMPLE_EDGE] llap
-                            SHUFFLE [RS_11]
-                              Select Operator [SEL_9] (rows=1 width=105)
-                                Output:["_col0","_col1","_col2","_col3"]
-                                Group By Operator [GBY_8] (rows=1 width=101)
-                                  Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
-                                <-Map 5 [SIMPLE_EDGE] llap
-                                  SHUFFLE [RS_7]
-                                    PartitionCols:_col0, _col1, _col2
-                                    Group By Operator [GBY_6] (rows=1 width=101)
-                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
-                                      Filter Operator [FIL_45] (rows=2 width=93)
-                                        predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0) and key is not null)
-                                        TableScan [TS_3] (rows=20 width=88)
-                                          default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
-                      <-Reducer 9 [SIMPLE_EDGE] llap
-                        SHUFFLE [RS_24]
-                          PartitionCols:_col0
-                          Select Operator [SEL_20] (rows=1 width=89)
-                            Output:["_col0","_col1"]
-                            Group By Operator [GBY_19] (rows=1 width=93)
-                              Output:["_col0","_col1","_col2"],keys:KEY._col0, KEY._col1, KEY._col2
-                            <-Map 8 [SIMPLE_EDGE] llap
-                              SHUFFLE [RS_18]
-                                PartitionCols:_col0, _col1, _col2
-                                Group By Operator [GBY_17] (rows=1 width=93)
-                                  Output:["_col0","_col1","_col2"],keys:key, c_int, c_float
-                                  Filter Operator [FIL_46] (rows=2 width=93)
-                                    predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0) and key is not null)
-                                    TableScan [TS_14] (rows=20 width=88)
-                                      default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
+            Select Operator [SEL_34] (rows=1 width=28)
+              Output:["_col0","_col1","_col2","_col3"]
+              Group By Operator [GBY_33] (rows=1 width=20)
+                Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
+              <-Reducer 2 [SIMPLE_EDGE] llap
+                SHUFFLE [RS_32]
+                  PartitionCols:_col0, _col1
+                  Group By Operator [GBY_31] (rows=1 width=20)
+                    Output:["_col0","_col1","_col2"],aggregations:["count()"],keys:_col1, _col4
+                    Select Operator [SEL_30] (rows=1 width=20)
+                      Output:["_col1","_col4"]
+                      Filter Operator [FIL_26] (rows=1 width=20)
+                        predicate:(((_col3 + _col6) >= 0) and ((UDFToLong(_col1) + _col4) >= 0) and ((_col1 >= 1) or (_col4 >= 1)) and ((_col3 > 0) or (_col1 >= 0)))
+                        Merge Join Operator [MERGEJOIN_48] (rows=3 width=20)
+                          Conds:RS_22._col0=RS_23._col0(Inner),RS_23._col0=RS_24._col0(Inner),Output:["_col1","_col3","_col4","_col6"]
+                        <-Map 1 [SIMPLE_EDGE] llap
+                          SHUFFLE [RS_22]
+                            PartitionCols:_col0
+                            Select Operator [SEL_2] (rows=18 width=84)
+                              Output:["_col0","_col1"]
+                              Filter Operator [FIL_45] (rows=18 width=84)
+                                predicate:((c_int > 0) and key is not null)
+                                TableScan [TS_0] (rows=20 width=84)
+                                  default@cbo_t3,cbo_t3,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int"]
+                        <-Reducer 7 [SIMPLE_EDGE] llap
+                          SHUFFLE [RS_23]
+                            PartitionCols:_col0
+                            Select Operator [SEL_12] (rows=1 width=97)
+                              Output:["_col0","_col1","_col2"]
+                            <-Reducer 6 [SIMPLE_EDGE] llap
+                              SHUFFLE [RS_11]
+                                Select Operator [SEL_9] (rows=1 width=105)
+                                  Output:["_col0","_col1","_col2","_col5"]
+                                  Group By Operator [GBY_8] (rows=1 width=101)
+                                    Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
+                                  <-Map 5 [SIMPLE_EDGE] llap
+                                    SHUFFLE [RS_7]
+                                      PartitionCols:_col0, _col1, _col2
+                                      Group By Operator [GBY_6] (rows=1 width=101)
+                                        Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
+                                        Filter Operator [FIL_46] (rows=2 width=93)
+                                          predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0) and key is not null)
+                                          TableScan [TS_3] (rows=20 width=88)
+                                            default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
+                        <-Reducer 9 [SIMPLE_EDGE] llap
+                          SHUFFLE [RS_24]
+                            PartitionCols:_col0
+                            Select Operator [SEL_20] (rows=1 width=89)
+                              Output:["_col0","_col1"]
+                              Group By Operator [GBY_19] (rows=1 width=93)
+                                Output:["_col0","_col1","_col2"],keys:KEY._col0, KEY._col1, KEY._col2
+                              <-Map 8 [SIMPLE_EDGE] llap
+                                SHUFFLE [RS_18]
+                                  PartitionCols:_col0, _col1, _col2
+                                  Group By Operator [GBY_17] (rows=1 width=93)
+                                    Output:["_col0","_col1","_col2"],keys:key, c_int, c_float
+                                    Filter Operator [FIL_47] (rows=2 width=93)
+                                      predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0) and key is not null)
+                                      TableScan [TS_14] (rows=20 width=88)
+                                        default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
 
 PREHOOK: query: explain select cbo_t3.c_int, c, count(*) from (select key as a, c_int+1 as b, sum(c_int) as c from cbo_t1 where (cbo_t1.c_int + 1 >= 0) and (cbo_t1.c_int > 0 or cbo_t1.c_float >= 0)  group by c_float, cbo_t1.c_int, key having cbo_t1.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0 order by b+c, a desc) cbo_t1 right outer join (select key as p, c_int+1 as q, sum(c_int) as r from cbo_t2 where (cbo_t2.c_int + 1 >= 0) and (cbo_t2.c_int > 0 or cbo_t2.c_float >= 0)  group by c_float, cbo_t2.c_int, key having cbo_t2.c_float > 0 and (c_int >=1 or c_float >= 1) and (c_int + c_float) >= 0) cbo_t2 on cbo_t1.a=p right outer join cbo_t3 on cbo_t1.a=key where (b + cbo_t2.q >= 2) and (b > 0 or c_int >= 0) group by cbo_t3.c_int, c
 PREHOOK: type: QUERY
@@ -560,7 +562,7 @@ Stage-0
                       <-Reducer 5 [SIMPLE_EDGE] llap
                         SHUFFLE [RS_11]
                           Select Operator [SEL_9] (rows=1 width=105)
-                            Output:["_col0","_col1","_col2","_col3"]
+                            Output:["_col0","_col1","_col2","_col5"]
                             Group By Operator [GBY_8] (rows=1 width=101)
                               Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                             <-Map 4 [SIMPLE_EDGE] llap
@@ -644,7 +646,7 @@ Stage-0
                           <-Reducer 9 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_22]
                               Select Operator [SEL_20] (rows=1 width=105)
-                                Output:["_col0","_col1","_col2","_col3"]
+                                Output:["_col0","_col1","_col2","_col5"]
                                 Group By Operator [GBY_19] (rows=1 width=101)
                                   Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                 <-Map 8 [SIMPLE_EDGE] llap
@@ -664,7 +666,7 @@ Stage-0
                           <-Reducer 6 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_11]
                               Select Operator [SEL_9] (rows=1 width=105)
-                                Output:["_col0","_col1","_col2","_col3"]
+                                Output:["_col0","_col1","_col2","_col5"]
                                 Group By Operator [GBY_8] (rows=1 width=101)
                                   Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                 <-Map 5 [SIMPLE_EDGE] llap
@@ -1226,10 +1228,10 @@ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:1
+    limit:-1
     Stage-1
       Reducer 3 llap
-      File Output Operator [FS_10]
+      File Output Operator [FS_11]
         Limit [LIM_9] (rows=1 width=97)
           Number of rows:1
           Select Operator [SEL_8] (rows=10 width=97)
@@ -1300,38 +1302,15 @@ POSTHOOK: query: explain select key from(select key from (select key from cbo_t1
 POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
-Vertex dependency in root stage
-Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
-
 Stage-0
   Fetch Operator
     limit:5
-    Stage-1
-      Reducer 3 llap
-      File Output Operator [FS_13]
-        Limit [LIM_12] (rows=5 width=85)
-          Number of rows:5
-          Limit [LIM_10] (rows=5 width=85)
-            Number of rows:5
-            Select Operator [SEL_9] (rows=5 width=85)
-              Output:["_col0"]
-            <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap
-              PARTITION_ONLY_SHUFFLE [RS_8]
-                Limit [LIM_7] (rows=5 width=85)
-                  Number of rows:5
-                  Limit [LIM_5] (rows=5 width=85)
-                    Number of rows:5
-                    Select Operator [SEL_4] (rows=5 width=85)
-                      Output:["_col0"]
-                    <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
-                      PARTITION_ONLY_SHUFFLE [RS_3]
-                        Limit [LIM_2] (rows=5 width=85)
-                          Number of rows:5
-                          Select Operator [SEL_1] (rows=20 width=80)
-                            Output:["_col0"]
-                            TableScan [TS_0] (rows=20 width=80)
-                              default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
+    Limit [LIM_2]
+      Number of rows:5
+      Select Operator [SEL_1]
+        Output:["_col0"]
+        TableScan [TS_0]
+          Output:["key"]
 
 PREHOOK: query: explain select key, c_int from(select key, c_int from (select key, c_int from cbo_t1 order by c_int limit 5)cbo_t1  order by c_int limit 5)cbo_t2  order by c_int limit 5
 PREHOOK: type: QUERY
@@ -1390,18 +1369,18 @@ Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
-    limit:5
+    limit:-1
     Stage-1
       Reducer 6 llap
-      File Output Operator [FS_45]
-        Limit [LIM_44] (rows=1 width=20)
+      File Output Operator [FS_46]
+        Limit [LIM_44] (rows=1 width=28)
           Number of rows:5
-          Select Operator [SEL_43] (rows=1 width=20)
+          Select Operator [SEL_43] (rows=1 width=28)
             Output:["_col0","_col1","_col2"]
           <-Reducer 5 [SIMPLE_EDGE] llap
             SHUFFLE [RS_42]
-              Select Operator [SEL_41] (rows=1 width=20)
-                Output:["_col0","_col1","_col2"]
+              Select Operator [SEL_41] (rows=1 width=28)
+                Output:["_col0","_col1","_col2","_col3"]
                 Group By Operator [GBY_40] (rows=1 width=20)
                   Output:["_col0","_col1","_col2"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1
                 <-Reducer 4 [SIMPLE_EDGE] llap
@@ -1413,14 +1392,14 @@ Stage-0
                         Output:["_col4","_col6"]
                         Filter Operator [FIL_35] (rows=3 width=20)
                           predicate:(((_col3 > 0) or (_col6 >= 0)) and ((_col3 + _col1) >= 0))
-                          Merge Join Operator [MERGEJOIN_57] (rows=7 width=20)
+                          Merge Join Operator [MERGEJOIN_58] (rows=7 width=20)
                             Conds:RS_31._col0=RS_32._col0(Inner),RS_32._col0=RS_33._col0(Inner),Output:["_col1","_col3","_col4","_col6"]
                           <-Map 10 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_33]
                               PartitionCols:_col0
                               Select Operator [SEL_30] (rows=18 width=84)
                                 Output:["_col0","_col1"]
-                                Filter Operator [FIL_56] (rows=18 width=84)
+                                Filter Operator [FIL_57] (rows=18 width=84)
                                   predicate:key is not null
                                   TableScan [TS_28] (rows=20 width=84)
                                     default@cbo_t3,cbo_t3,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int"]
@@ -1436,7 +1415,7 @@ Stage-0
                                   <-Reducer 2 [SIMPLE_EDGE] llap
                                     SHUFFLE [RS_8]
                                       Select Operator [SEL_6] (rows=3 width=105)
-                                        Output:["_col0","_col1","_col2","_col3"]
+                                        Output:["_col0","_col1","_col2","_col5"]
                                         Group By Operator [GBY_5] (rows=3 width=101)
                                           Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                         <-Map 1 [SIMPLE_EDGE] llap
@@ -1444,7 +1423,7 @@ Stage-0
                                             PartitionCols:_col0, _col1, _col2
                                             Group By Operator [GBY_3] (rows=3 width=101)
                                               Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
-                                              Filter Operator [FIL_53] (rows=6 width=93)
+                                              Filter Operator [FIL_54] (rows=6 width=93)
                                                 predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)))
                                                 TableScan [TS_0] (rows=20 width=88)
                                                   default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
@@ -1468,7 +1447,7 @@ Stage-0
                                             PartitionCols:_col0, _col1, _col2
                                             Group By Operator [GBY_17] (rows=3 width=101)
                                               Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
-                                              Filter Operator [FIL_55] (rows=6 width=93)
+                                              Filter Operator [FIL_56] (rows=6 width=93)
                                                 predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)))
                                                 TableScan [TS_14] (rows=20 width=88)
                                                   default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
@@ -1603,12 +1582,12 @@ Stage-0
                     <-Map 10 [SIMPLE_EDGE] llap
                       SHUFFLE [RS_32]
                         PartitionCols:_col0
-                        Group By Operator [GBY_29] (rows=3 width=85)
+                        Group By Operator [GBY_29] (rows=6 width=85)
                           Output:["_col0"],keys:_col0
-                          Select Operator [SEL_25] (rows=6 width=85)
+                          Select Operator [SEL_25] (rows=18 width=80)
                             Output:["_col0"]
-                            Filter Operator [FIL_50] (rows=6 width=85)
-                              predicate:(UDFToDouble(key) > 0.0)
+                            Filter Operator [FIL_50] (rows=18 width=80)
+                              predicate:key is not null
                               TableScan [TS_23] (rows=20 width=80)
                                 default@cbo_t3,cbo_t3,Tbl:COMPLETE,Col:COMPLETE,Output:["key"]
                     <-Reducer 3 [SIMPLE_EDGE] llap
@@ -1618,8 +1597,8 @@ Stage-0
                           Output:["_col0","_col1"]
                         <-Reducer 2 [SIMPLE_EDGE] llap
                           SHUFFLE [RS_9]
-                            Select Operator [SEL_8] (rows=1 width=101)
-                              Output:["_col0","_col1","_col2"]
+                            Select Operator [SEL_8] (rows=1 width=109)
+                              Output:["_col0","_col1","_col4"]
                               Filter Operator [FIL_7] (rows=1 width=101)
                                 predicate:(((UDFToDouble(_col2) >= 1.0) or (_col3 >= 1)) and ((UDFToDouble(_col2) + UDFToDouble(_col3)) >= 0.0))
                                 Select Operator [SEL_6] (rows=1 width=101)
@@ -1645,7 +1624,7 @@ Stage-0
                           <-Reducer 8 [SIMPLE_EDGE] llap
                             SHUFFLE [RS_20]
                               Select Operator [SEL_18] (rows=1 width=93)
-                                Output:["_col0","_col1"]
+                                Output:["_col0","_col3"]
                                 Group By Operator [GBY_17] (rows=1 width=101)
                                   Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                 <-Map 7 [SIMPLE_EDGE] llap
@@ -1653,8 +1632,8 @@ Stage-0
                                     PartitionCols:_col0, _col1, _col2
                                     Group By Operator [GBY_15] (rows=1 width=101)
                                       Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
-                                      Filter Operator [FIL_49] (rows=1 width=93)
-                                        predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0) and (UDFToDouble(key) > 0.0))
+                                      Filter Operator [FIL_49] (rows=2 width=93)
+                                        predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0)) and (c_float > 0) and ((c_int >= 1) or (c_float >= 1)) and ((UDFToFloat(c_int) + c_float) >= 0) and key is not null)
                                         TableScan [TS_12] (rows=20 width=88)
                                           default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/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 0a8df61..57594e0 100644
--- a/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
+++ b/ql/src/test/results/clientpositive/llap/limit_pushdown.q.out
@@ -1341,11 +1341,11 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col1 (type: double)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: double)
                     sort order: +
-                    Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 2.0E-5
         Reducer 3 
             Execution mode: llap
@@ -1367,7 +1367,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 100
+      limit: -1
       Processor Tree:
         ListSink
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out b/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out
index 24645b6..351ee01 100644
--- a/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out
+++ b/ql/src/test/results/clientpositive/llap/limit_pushdown3.q.out
@@ -1283,11 +1283,11 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col1 (type: double)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: double)
                     sort order: +
-                    Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 2.0E-5
         Reducer 3 
             Execution mode: llap
@@ -1309,7 +1309,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 100
+      limit: -1
       Processor Tree:
         ListSink
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out b/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out
index 77062c7..c89ca6b 100644
--- a/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out
+++ b/ql/src/test/results/clientpositive/llap/offset_limit_ppd_optimizer.q.out
@@ -1166,11 +1166,11 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col1 (type: double)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: double)
                     sort order: +
-                    Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 500 Data size: 96000 Basic stats: COMPLETE Column stats: COMPLETE
                     TopN Hash Memory Usage: 2.0E-5
         Reducer 3 
             Execution mode: llap
@@ -1193,7 +1193,7 @@ STAGE PLANS:
 
   Stage: Stage-0
     Fetch Operator
-      limit: 70
+      limit: -1
       Processor Tree:
         ListSink
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/llap/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in.q.out b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
index d7fd29e..1f9c9e4 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
@@ -3315,13 +3315,13 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 223 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col1 (type: string), _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 1 Data size: 223 Basic stats: COMPLETE Column stats: COMPLETE
+                  expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
+                  outputColumnNames: _col1, _col3, _col4
+                  Statistics: Num rows: 1 Data size: 325 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: _col0 (type: string), _col2 (type: int)
+                    key expressions: _col3 (type: string), _col4 (type: int)
                     sort order: ++
-                    Statistics: Num rows: 1 Data size: 223 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 325 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: string)
         Reducer 3 
             Execution mode: llap

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out b/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
index 8402104..d6865dd 100644
--- a/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_coalesce.q.out
@@ -65,14 +65,18 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                    projectedOutputColumns: [5, 0, 1, 2, 3, 4]
-                    selectExpressions: ConstantVectorExpression(val null) -> 5:double
+                    projectedOutputColumns: [0, 1, 2, 3, 4]
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                    File Sink Vectorization:
-                        className: VectorFileSinkOperator
-                        native: false
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumns: [5, 0, 1, 2, 3, 4]
+                        selectExpressions: ConstantVectorExpression(val null) -> 5:double
+                      File Sink Vectorization:
+                          className: VectorFileSinkOperator
+                          native: false
 
   Stage: Stage-0
     Fetch Operator
@@ -170,14 +174,18 @@ STAGE PLANS:
                 Select Vectorization:
                     className: VectorSelectOperator
                     native: true
-                    projectedOutputColumns: [3, 0, 1, 2]
-                    selectExpressions: ConstantVectorExpression(val null) -> 3:tinyint
+                    projectedOutputColumns: [0, 1, 2]
                   Limit Vectorization:
                       className: VectorLimitOperator
                       native: true
-                    File Sink Vectorization:
-                        className: VectorFileSinkOperator
-                        native: false
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumns: [3, 0, 1, 2]
+                        selectExpressions: ConstantVectorExpression(val null) -> 3:tinyint
+                      File Sink Vectorization:
+                          className: VectorFileSinkOperator
+                          native: false
 
   Stage: Stage-0
     Fetch Operator
@@ -231,8 +239,6 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
-      Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -246,11 +252,14 @@ STAGE PLANS:
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                          projectedOutputColumns: []
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkEmptyKeyOperator
+                          projectedOutputColumns: [12, 13, 14]
+                          selectExpressions: ConstantVectorExpression(val null) -> 12:float, ConstantVectorExpression(val null) -> 13:bigint, ConstantVectorExpression(val 0.0) -> 14:double
+                        Limit Vectorization:
+                            className: VectorLimitOperator
                             native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          File Sink Vectorization:
+                              className: VectorFileSinkOperator
+                              native: false
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -258,30 +267,9 @@ STAGE PLANS:
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 groupByVectorOutput: true
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: true
-                usesVectorUDFAdaptor: false
-                vectorized: true
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                groupByVectorOutput: true
                 allNative: false
                 usesVectorUDFAdaptor: false
                 vectorized: true
-            Reduce Operator Tree:
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumns: [0, 1, 2]
-                    selectExpressions: ConstantVectorExpression(val null) -> 0:float, ConstantVectorExpression(val null) -> 1:bigint, ConstantVectorExpression(val 0.0) -> 2:double
-                  Limit Vectorization:
-                      className: VectorLimitOperator
-                      native: true
-                    File Sink Vectorization:
-                        className: VectorFileSinkOperator
-                        native: false
 
   Stage: Stage-0
     Fetch Operator
@@ -439,8 +427,6 @@ STAGE DEPENDENCIES:
 STAGE PLANS:
   Stage: Stage-1
     Tez
-      Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -454,11 +440,14 @@ STAGE PLANS:
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                          projectedOutputColumns: []
-                        Reduce Sink Vectorization:
-                            className: VectorReduceSinkEmptyKeyOperator
+                          projectedOutputColumns: [12, 13, 14]
+                          selectExpressions: ConstantVectorExpression(val null) -> 12:float, ConstantVectorExpression(val null) -> 13:bigint, ConstantVectorExpression(val null) -> 14:float
+                        Limit Vectorization:
+                            className: VectorLimitOperator
                             native: true
-                            nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          File Sink Vectorization:
+                              className: VectorFileSinkOperator
+                              native: false
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -466,30 +455,9 @@ STAGE PLANS:
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 groupByVectorOutput: true
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: true
-                usesVectorUDFAdaptor: false
-                vectorized: true
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                groupByVectorOutput: true
                 allNative: false
                 usesVectorUDFAdaptor: false
                 vectorized: true
-            Reduce Operator Tree:
-                Select Vectorization:
-                    className: VectorSelectOperator
-                    native: true
-                    projectedOutputColumns: [0, 1, 2]
-                    selectExpressions: ConstantVectorExpression(val null) -> 0:float, ConstantVectorExpression(val null) -> 1:bigint, ConstantVectorExpression(val null) -> 2:float
-                  Limit Vectorization:
-                      className: VectorLimitOperator
-                      native: true
-                    File Sink Vectorization:
-                        className: VectorFileSinkOperator
-                        native: false
 
   Stage: Stage-0
     Fetch Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/a74c1e7c/ql/src/test/results/clientpositive/llap/vector_date_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_date_1.q.out b/ql/src/test/results/clientpositive/llap/vector_date_1.q.out
index a4f1050..e8f32af 100644
--- a/ql/src/test/results/clientpositive/llap/vector_date_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_date_1.q.out
@@ -632,9 +632,6 @@ STAGE PLANS:
   Stage: Stage-1
     Tez
 #### A masked pattern was here ####
-      Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
-#### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
@@ -645,29 +642,18 @@ STAGE PLANS:
                     predicate: ((dt1 = 2001-01-01) and (2001-01-01 = dt1) and (dt1 <> 1970-01-01) and (1970-01-01 <> dt1) and (dt1 > 1970-01-01) and (dt1 >= 1970-01-01) and (1970-01-01 < dt1) and (1970-01-01 <= dt1)) (type: boolean)
                     Statistics: Num rows: 1 Data size: 74 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      expressions: dt2 (type: date)
-                      outputColumnNames: _col1
+                      expressions: 2001-01-01 (type: date), dt2 (type: date)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 74 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        sort order: 
+                      File Output Operator
+                        compressed: false
                         Statistics: Num rows: 1 Data size: 74 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: date)
+                        table:
+                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Reducer 2 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: 2001-01-01 (type: date), VALUE._col0 (type: date)
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 74 Basic stats: COMPLETE Column stats: NONE
-                File Output Operator
-                  compressed: false
-                  Statistics: Num rows: 1 Data size: 74 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator