You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2015/11/20 22:46:48 UTC

[08/12] hive git commit: HIVE-11684: Implement limit pushdown through outer join in CBO (Jesus Camacho Rodriguez, reviewed by Laljo John Pullokkaran)

HIVE-11684: Implement limit pushdown through outer join in CBO (Jesus Camacho Rodriguez, reviewed by Laljo John Pullokkaran)


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

Branch: refs/heads/master-fixed
Commit: 39bd58b2f6c1614bc8f2eb9a28e0d039c7df2f73
Parents: 2d25cf2
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Mon Nov 16 11:32:21 2015 +0100
Committer: Owen O'Malley <om...@apache.org>
Committed: Fri Nov 20 13:46:23 2015 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  11 +
 .../test/resources/testconfiguration.properties |   1 +
 .../calcite/reloperators/HiveSortLimit.java     |  15 +-
 .../rules/HiveProjectSortTransposeRule.java     |  66 ++
 .../calcite/rules/HiveSortJoinReduceRule.java   | 135 +++
 .../calcite/rules/HiveSortMergeRule.java        |  78 ++
 .../rules/HiveSortProjectTransposeRule.java     |  79 ++
 .../calcite/rules/HiveSortRemoveRule.java       |  79 ++
 .../calcite/stats/HiveRelMdRowCount.java        |  18 +
 .../calcite/stats/HiveRelMdSelectivity.java     |  10 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  32 +-
 .../clientpositive/limit_join_transpose.q       |  99 +++
 .../clientpositive/tez_dynpart_hashjoin_3.q     |  24 +
 .../clientpositive/join32_lessSize.q.out        |  46 +-
 .../results/clientpositive/join_merging.q.out   |  34 +-
 .../clientpositive/limit_join_transpose.q.out   | 869 +++++++++++++++++++
 .../clientpositive/spark/join32_lessSize.q.out  | 122 +--
 .../clientpositive/spark/join_merging.q.out     |  76 +-
 .../results/clientpositive/subquery_notin.q.out | 328 ++++---
 .../subquery_notin_having.q.java1.7.out         |  94 +-
 .../clientpositive/tez/explainuser_1.q.out      | 171 ++--
 .../tez/tez_dynpart_hashjoin_3.q.out            | 200 +++++
 22 files changed, 2137 insertions(+), 450 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 4f8209a..09f3c3e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1311,6 +1311,17 @@ public class HiveConf extends Configuration {
         "we are increasing the number of files possibly by a big margin. So, we merge aggressively."),
     HIVEOPTCORRELATION("hive.optimize.correlation", false, "exploit intra-query correlations."),
 
+    HIVE_OPTIMIZE_LIMIT_JOIN_TRANSPOSE("hive.optimize.limitjointranspose", false,
+        "Whether to push a limit through left/right outer join. If the value is true and the size of the outer\n" +
+        "input is reduced enough (as specified in hive.optimize.limitjointranspose.reduction), the limit is pushed\n" +
+        "to the outer input; to remain semantically correct, the limit is kept on top of the join too."),
+    HIVE_OPTIMIZE_LIMIT_JOIN_TRANSPOSE_REDUCTION_PERCENTAGE("hive.optimize.limitjointranspose.reductionpercentage", 1.0f,
+        "When hive.optimize.limitjointranspose is true, this variable specifies the minimal reduction of the\n" +
+        "size of the outer input of the join that we should get in order to apply the rule."),
+    HIVE_OPTIMIZE_LIMIT_JOIN_TRANSPOSE_REDUCTION_TUPLES("hive.optimize.limitjointranspose.reductiontuples", (long) 0,
+        "When hive.optimize.limitjointranspose is true, this variable specifies the minimal reduction in the\n" +
+        "number of tuples of the outer input of the join that you should get in order to apply the rule."),
+
     HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME("hive.optimize.skewjoin.compiletime", false,
         "Whether to create a separate plan for skewed keys for the tables in the join.\n" +
         "This is based on the skewed keys stored in the metadata. At compile time, the plan is broken\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index b86dd19..0c5140c 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -388,6 +388,7 @@ minitez.query.files=bucket_map_join_tez1.q,\
   tez_insert_overwrite_local_directory_1.q,\
   tez_dynpart_hashjoin_1.q,\
   tez_dynpart_hashjoin_2.q,\
+  tez_dynpart_hashjoin_3.q,\
   tez_vector_dynpart_hashjoin_1.q,\
   tez_vector_dynpart_hashjoin_2.q,\
   tez_join_hash.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java
index 82fdc0e..3077177 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortLimit.java
@@ -44,6 +44,8 @@ public class HiveSortLimit extends Sort implements HiveRelNode {
   // 4. This is used by ASTConverter after we are done with Calcite Planning
   private ImmutableMap<Integer, RexNode> mapOfInputRefToRexCall;
 
+  private boolean ruleCreated;
+
   public HiveSortLimit(RelOptCluster cluster, RelTraitSet traitSet, RelNode child,
       RelCollation collation, RexNode offset, RexNode fetch) {
     super(cluster, TraitsUtil.getSortTraitSet(cluster, traitSet, collation), child, collation,
@@ -74,7 +76,10 @@ public class HiveSortLimit extends Sort implements HiveRelNode {
     // TODO: can we blindly copy sort trait? What if inputs changed and we
     // are now sorting by different cols
     RelCollation canonizedCollation = traitSet.canonize(newCollation);
-    return new HiveSortLimit(getCluster(), traitSet, newInput, canonizedCollation, offset, fetch);
+    HiveSortLimit sortLimit =
+            new HiveSortLimit(getCluster(), traitSet, newInput, canonizedCollation, offset, fetch);
+    sortLimit.setRuleCreated(ruleCreated);
+    return sortLimit;
   }
 
   public RexNode getFetchExpr() {
@@ -93,6 +98,14 @@ public class HiveSortLimit extends Sort implements HiveRelNode {
   public void implement(Implementor implementor) {
   }
 
+  public boolean isRuleCreated() {
+    return ruleCreated;
+  }
+
+  public void setRuleCreated(boolean ruleCreated) {
+    this.ruleCreated = ruleCreated;
+  }
+
   private static class HiveSortRelFactory implements RelFactories.SortFactory {
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/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
new file mode 100644
index 0000000..9d74b19
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
@@ -0,0 +1,66 @@
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+
+import com.google.common.collect.ImmutableList;
+
+public class HiveProjectSortTransposeRule extends RelOptRule {
+
+  public static final HiveProjectSortTransposeRule INSTANCE =
+      new HiveProjectSortTransposeRule();
+
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Creates a HiveProjectSortTransposeRule.
+   */
+  private HiveProjectSortTransposeRule() {
+    super(
+        operand(
+            HiveProject.class,
+            operand(HiveSortLimit.class, any())));
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  // implement RelOptRule
+  public void onMatch(RelOptRuleCall call) {
+    final HiveProject project = call.rel(0);
+    final HiveSortLimit sort = call.rel(1);
+
+    // Determine mapping between project input and output fields. If sort
+    // relies on non-trivial expressions, we can't push.
+    final Mappings.TargetMapping map =
+        RelOptUtil.permutation(
+            project.getProjects(), project.getInput().getRowType()).inverse();
+    for (RelFieldCollation fc : sort.getCollation().getFieldCollations()) {
+      if (map.getTarget(fc.getFieldIndex()) < 0) {
+        return;
+      }
+    }
+
+    // Create new collation
+    final RelCollation newCollation =
+        RelCollationTraitDef.INSTANCE.canonize(
+            RexUtil.apply(map, sort.getCollation()));
+
+    // New operators
+    final RelNode newProject = project.copy(sort.getInput().getTraitSet(),
+            ImmutableList.<RelNode>of(sort.getInput()));
+    final HiveSortLimit newSort = sort.copy(newProject.getTraitSet(),
+            newProject, newCollation, sort.offset, sort.fetch);
+
+    call.transformTo(newSort);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortJoinReduceRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortJoinReduceRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortJoinReduceRule.java
new file mode 100644
index 0000000..d8b9c54
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortJoinReduceRule.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+
+/**
+ * Planner rule that pushes
+ * a {@link org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit}
+ * past a {@link org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin}.
+ */
+public class HiveSortJoinReduceRule extends RelOptRule {
+
+  public static final HiveSortJoinReduceRule INSTANCE =
+          new HiveSortJoinReduceRule();
+
+  //~ Constructors -----------------------------------------------------------
+
+  private HiveSortJoinReduceRule() {
+    super(
+        operand(
+            HiveSortLimit.class,
+            operand(HiveJoin.class, any())));
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final HiveSortLimit sortLimit = call.rel(0);
+    final HiveJoin join = call.rel(1);
+
+    // If sort does not contain a limit operation, we bail out
+    if (!HiveCalciteUtil.limitRelNode(sortLimit)) {
+      return false;
+    }
+
+    // 1) If join is not a left or right outer, we bail out
+    // 2) If any sort column is not part of the input where the
+    // sort is pushed, we bail out
+    RelNode reducedInput;
+    if (join.getJoinType() == JoinRelType.LEFT) {
+      reducedInput = join.getLeft();
+      if (sortLimit.getCollation() != RelCollations.EMPTY) {
+        for (RelFieldCollation relFieldCollation
+            : sortLimit.getCollation().getFieldCollations()) {
+          if (relFieldCollation.getFieldIndex()
+              >= join.getLeft().getRowType().getFieldCount()) {
+            return false;
+          }
+        }
+      }
+    } else if (join.getJoinType() == JoinRelType.RIGHT) {
+      reducedInput = join.getRight();
+      if (sortLimit.getCollation() != RelCollations.EMPTY) {
+        for (RelFieldCollation relFieldCollation
+            : sortLimit.getCollation().getFieldCollations()) {
+          if (relFieldCollation.getFieldIndex()
+              < join.getLeft().getRowType().getFieldCount()) {
+            return false;
+          }
+        }
+      }
+    } else {
+      return false;
+    }
+
+    // Finally, if we do not reduce the input size, we bail out
+    if (RexLiteral.intValue(sortLimit.fetch)
+            >= RelMetadataQuery.getRowCount(reducedInput)) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final HiveSortLimit sortLimit = call.rel(0);
+    final HiveJoin join = call.rel(1);
+    RelNode inputLeft = join.getLeft();
+    RelNode inputRight = join.getRight();
+
+    // We create a new sort operator on the corresponding input
+    if (join.getJoinType() == JoinRelType.LEFT) {
+      inputLeft = sortLimit.copy(sortLimit.getTraitSet(), inputLeft,
+              sortLimit.getCollation(), sortLimit.offset, sortLimit.fetch);
+      ((HiveSortLimit) inputLeft).setRuleCreated(true);
+    } else {
+      // Adjust right collation
+      final RelCollation rightCollation =
+              RelCollationTraitDef.INSTANCE.canonize(
+                  RelCollations.shift(sortLimit.getCollation(),
+                      -join.getLeft().getRowType().getFieldCount()));
+      inputRight = sortLimit.copy(sortLimit.getTraitSet().replace(rightCollation), inputRight,
+              rightCollation, sortLimit.offset, sortLimit.fetch);
+      ((HiveSortLimit) inputRight).setRuleCreated(true);
+    }
+    // We copy the join and the top sort operator
+    RelNode result = join.copy(join.getTraitSet(), join.getCondition(), inputLeft,
+            inputRight, join.getJoinType(), join.isSemiJoinDone());
+    result = sortLimit.copy(sortLimit.getTraitSet(), result, sortLimit.getCollation(),
+            sortLimit.offset, sortLimit.fetch);
+
+    call.transformTo(result);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortMergeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortMergeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortMergeRule.java
new file mode 100644
index 0000000..ebb20c2
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortMergeRule.java
@@ -0,0 +1,78 @@
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+
+/**
+ * This rule will merge two HiveSortLimit operators.
+ * 
+ * It is applied when the top match is a pure limit operation (no sorting).
+ * 
+ * If the bottom operator is not synthetic and does not contain a limit,
+ * we currently bail out. Thus, we avoid a lot of unnecessary limit operations
+ * in the middle of the execution plan that could create performance regressions.
+ */
+public class HiveSortMergeRule extends RelOptRule {
+
+  public static final HiveSortMergeRule INSTANCE =
+      new HiveSortMergeRule();
+
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Creates a HiveSortProjectTransposeRule.
+   */
+  private HiveSortMergeRule() {
+    super(
+        operand(
+            HiveSortLimit.class,
+            operand(HiveSortLimit.class, any())));
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final HiveSortLimit topSortLimit = call.rel(0);
+    final HiveSortLimit bottomSortLimit = call.rel(1);
+
+    // If top operator is not a pure limit, we bail out
+    if (!HiveCalciteUtil.pureLimitRelNode(topSortLimit)) {
+      return false;
+    }
+
+    // If the bottom operator is not synthetic and it does not contain a limit,
+    // we will bail out; we do not want to end up with limits all over the tree
+    if (topSortLimit.isRuleCreated() && !bottomSortLimit.isRuleCreated() &&
+            bottomSortLimit.fetch == null) {
+      return false;
+    }
+
+    return true;
+  }
+
+  // implement RelOptRule
+  public void onMatch(RelOptRuleCall call) {
+    final HiveSortLimit topSortLimit = call.rel(0);
+    final HiveSortLimit bottomSortLimit = call.rel(1);
+
+    // Lowest limit
+    final RexNode newLimit;
+    if (bottomSortLimit.fetch != null && RexLiteral.intValue(topSortLimit.fetch)
+            >= RexLiteral.intValue(bottomSortLimit.fetch)) {
+      newLimit = bottomSortLimit.fetch;
+    } else {
+      newLimit = topSortLimit.fetch;
+    }
+
+    final HiveSortLimit newSort = bottomSortLimit.copy(bottomSortLimit.getTraitSet(),
+            bottomSortLimit.getInput(), bottomSortLimit.collation, null, newLimit);
+
+    call.transformTo(newSort);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
new file mode 100644
index 0000000..debfbe0
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
@@ -0,0 +1,79 @@
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexUtil;
+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.HiveSortLimit;
+
+import com.google.common.collect.ImmutableList;
+
+public class HiveSortProjectTransposeRule extends RelOptRule {
+
+  public static final HiveSortProjectTransposeRule INSTANCE =
+      new HiveSortProjectTransposeRule();
+
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Creates a HiveSortProjectTransposeRule.
+   */
+  private HiveSortProjectTransposeRule() {
+    super(
+        operand(
+            HiveSortLimit.class,
+            operand(HiveProject.class, any())));
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final HiveSortLimit sortLimit = call.rel(0);
+
+    // If does not contain a limit operation, we bail out
+    if (!HiveCalciteUtil.limitRelNode(sortLimit)) {
+      return false;
+    }
+
+    return true;
+  }
+
+  // implement RelOptRule
+  public void onMatch(RelOptRuleCall call) {
+    final HiveSortLimit sort = call.rel(0);
+    final HiveProject project = call.rel(1);
+
+    // Determine mapping between project input and output fields. If sort
+    // relies on non-trivial expressions, we can't push.
+    final Mappings.TargetMapping map =
+        RelOptUtil.permutation(
+            project.getProjects(), project.getInput().getRowType());
+    for (RelFieldCollation fc : sort.getCollation().getFieldCollations()) {
+      if (map.getTargetOpt(fc.getFieldIndex()) < 0) {
+        return;
+      }
+    }
+
+    // Create new collation
+    final RelCollation newCollation =
+        RelCollationTraitDef.INSTANCE.canonize(
+            RexUtil.apply(map, sort.getCollation()));
+
+    // New operators
+    final HiveSortLimit newSort = sort.copy(sort.getTraitSet().replace(newCollation),
+            project.getInput(), newCollation, sort.offset, sort.fetch);
+    final RelNode newProject = project.copy(sort.getTraitSet(),
+            ImmutableList.<RelNode>of(newSort));
+
+    call.transformTo(newProject);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortRemoveRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortRemoveRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortRemoveRule.java
new file mode 100644
index 0000000..618c717
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortRemoveRule.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+
+/**
+ * Planner rule that removes
+ * a {@link org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit}.
+ */
+public class HiveSortRemoveRule extends RelOptRule {
+
+  protected final float reductionProportion;
+  protected final float reductionTuples;
+
+  //~ Constructors -----------------------------------------------------------
+
+  public HiveSortRemoveRule(float reductionProportion, long reductionTuples) {
+    this(operand(HiveSortLimit.class, any()), reductionProportion, reductionTuples);
+  }
+
+  private HiveSortRemoveRule(RelOptRuleOperand operand, float reductionProportion,
+          long reductionTuples) {
+    super(operand);
+    this.reductionProportion = reductionProportion;
+    this.reductionTuples = reductionTuples;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final HiveSortLimit sortLimit = call.rel(0);
+
+    // If it is not created by HiveSortJoinReduceRule, we cannot remove it
+    if (!sortLimit.isRuleCreated()) {
+      return false;
+    }
+
+    // Finally, if we do not reduce the size input enough, we bail out
+    int limit = RexLiteral.intValue(sortLimit.fetch);
+    Double rowCount = RelMetadataQuery.getRowCount(sortLimit.getInput());
+    if (rowCount != null && limit <= reductionProportion * rowCount &&
+            rowCount - limit >= reductionTuples) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final HiveSortLimit sortLimit = call.rel(0);
+
+    // We remove the limit operator
+    call.transformTo(sortLimit.getInput());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
index 728c5aa..caf8978 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.SemiJoin;
+import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMdRowCount;
@@ -38,6 +39,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -87,6 +89,22 @@ public class HiveRelMdRowCount extends RelMdRowCount {
     return super.getRowCount(rel);
   }
 
+  @Override
+  public Double getRowCount(Sort rel) {
+    final Double rowCount = RelMetadataQuery.getRowCount(rel.getInput());
+    if (rowCount != null && rel.fetch != null) {
+      final int offset = rel.offset == null ? 0 : RexLiteral.intValue(rel.offset);
+      final int limit = RexLiteral.intValue(rel.fetch);
+      final Double offsetLimit = new Double(offset + limit);
+      // offsetLimit is smaller than rowCount of the input operator
+      // thus, we return the offsetLimit
+      if (offsetLimit < rowCount) {
+        return offsetLimit;
+      }
+    }
+    return rowCount;
+  }
+
   static class PKFKRelationInfo {
     public final int fkSide;
     public final double ndvScalingFactor;

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
index 715f24f..a0eb83d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
@@ -61,6 +61,16 @@ public class HiveRelMdSelectivity extends RelMdSelectivity {
   public Double getSelectivity(HiveJoin j, RexNode predicate) throws CalciteSemanticException {
     if (j.getJoinType().equals(JoinRelType.INNER)) {
       return computeInnerJoinSelectivity(j, predicate);
+    } else if (j.getJoinType().equals(JoinRelType.LEFT) ||
+            j.getJoinType().equals(JoinRelType.RIGHT)) {
+      double left = RelMetadataQuery.getRowCount(j.getLeft());
+      double right = RelMetadataQuery.getRowCount(j.getRight());
+      double product = left * right;
+      double innerJoinSelectivity = computeInnerJoinSelectivity(j, predicate);
+      if (j.getJoinType().equals(JoinRelType.LEFT)) {
+        return Math.max(innerJoinSelectivity, left/product);
+      }
+      return Math.max(innerJoinSelectivity, right/product);
     }
     return 1.0;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/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 de67b54..bfe4633 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
@@ -148,8 +148,13 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinToMultiJoinRule
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePartitionPruneRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePreFilteringRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectMergeRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectSortTransposeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRelFieldTrimmer;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRulesRegistry;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortJoinReduceRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortMergeRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortProjectTransposeRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortRemoveRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveWindowingFixRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.HiveOpConverter;
@@ -953,16 +958,31 @@ public class CalcitePlanner extends SemanticAnalyzer {
         basePlan = hepPlan(basePlan, true, mdProvider, HiveExpandDistinctAggregatesRule.INSTANCE);
       }
 
-      // 1. Push Down Semi Joins
+      // 1. Push down limit through outer join
+      if (conf.getBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_LIMIT_JOIN_TRANSPOSE)) {
+        // This should be a cost based decision, but till we enable the extended cost
+        // model, we will use the given value for the variable
+        final float reductionProportion = HiveConf.getFloatVar(conf,
+            HiveConf.ConfVars.HIVE_OPTIMIZE_LIMIT_JOIN_TRANSPOSE_REDUCTION_PERCENTAGE);
+        final long reductionTuples = HiveConf.getLongVar(conf,
+            HiveConf.ConfVars.HIVE_OPTIMIZE_LIMIT_JOIN_TRANSPOSE_REDUCTION_TUPLES);
+        basePlan = hepPlan(basePlan, true, mdProvider, HiveSortMergeRule.INSTANCE,
+            HiveSortProjectTransposeRule.INSTANCE, HiveSortJoinReduceRule.INSTANCE);
+        basePlan = hepPlan(basePlan, true, mdProvider, HepMatchOrder.BOTTOM_UP,
+            new HiveSortRemoveRule(reductionProportion, reductionTuples),
+            HiveProjectSortTransposeRule.INSTANCE);
+      }
+
+      // 2. Push Down Semi Joins
       basePlan = hepPlan(basePlan, true, mdProvider, SemiJoinJoinTransposeRule.INSTANCE,
           SemiJoinFilterTransposeRule.INSTANCE, SemiJoinProjectTransposeRule.INSTANCE);
 
-      // 2. Add not null filters
+      // 3. Add not null filters
       if (conf.getBoolVar(HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP)) {
         basePlan = hepPlan(basePlan, true, mdProvider, HiveJoinAddNotNullRule.INSTANCE);
       }
 
-      // 3. Constant propagation, common filter extraction, and PPD
+      // 4. Constant propagation, common filter extraction, and PPD
       basePlan = hepPlan(basePlan, true, mdProvider,
           ReduceExpressionsRule.PROJECT_INSTANCE,
           ReduceExpressionsRule.FILTER_INSTANCE,
@@ -976,12 +996,12 @@ public class CalcitePlanner extends SemanticAnalyzer {
           new FilterAggregateTransposeRule(Filter.class,
               HiveFilter.DEFAULT_FILTER_FACTORY, Aggregate.class));
 
-      // 4. Transitive inference & Partition Pruning
+      // 5. Transitive inference & Partition Pruning
       basePlan = hepPlan(basePlan, false, mdProvider, new HiveJoinPushTransitivePredicatesRule(
           Join.class, HiveFilter.DEFAULT_FILTER_FACTORY),
           new HivePartitionPruneRule(conf));
 
-      // 5. Projection Pruning
+      // 6. Projection Pruning
       HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null,
           cluster, HiveProject.DEFAULT_PROJECT_FACTORY,
           HiveFilter.DEFAULT_FILTER_FACTORY, HiveJoin.HIVE_JOIN_FACTORY,
@@ -989,7 +1009,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
           HiveAggregate.HIVE_AGGR_REL_FACTORY, HiveUnion.UNION_REL_FACTORY);
       basePlan = fieldTrimmer.trim(basePlan);
 
-      // 6. Rerun PPD through Project as column pruning would have introduced DT
+      // 7. Rerun PPD through Project as column pruning would have introduced DT
       // above scans
       basePlan = hepPlan(basePlan, true, mdProvider,
           new FilterProjectTransposeRule(Filter.class, HiveFilter.DEFAULT_FILTER_FACTORY,

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/test/queries/clientpositive/limit_join_transpose.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/limit_join_transpose.q b/ql/src/test/queries/clientpositive/limit_join_transpose.q
new file mode 100644
index 0000000..bfc2378
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/limit_join_transpose.q
@@ -0,0 +1,99 @@
+set hive.optimize.limitjointranspose=false;
+
+explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1;
+
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1;
+
+
+set hive.optimize.limitjointranspose=true;
+set hive.optimize.limitjointranspose.reductionpercentage=0.0001f;
+set hive.optimize.limitjointranspose.reductiontuples=10;
+
+explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1;
+
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1;
+
+
+set hive.optimize.limitjointranspose.reductionpercentage=0.1f;
+set hive.optimize.limitjointranspose.reductiontuples=10;
+
+explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1;
+
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1;
+
+explain
+select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1;
+
+select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1;
+
+set hive.optimize.limitjointranspose.reductionpercentage=1f;
+set hive.optimize.limitjointranspose.reductiontuples=0;
+
+explain
+select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1;
+
+select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1;
+
+explain
+select *
+from src src1 right outer join (
+  select src2.key, src2.value
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+order by src2.key
+limit 1;
+
+select *
+from src src1 right outer join (
+  select src2.key, src2.value
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+order by src2.key
+limit 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_3.q b/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_3.q
new file mode 100644
index 0000000..1994b40
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_3.q
@@ -0,0 +1,24 @@
+set hive.optimize.limitjointranspose=true;
+set hive.optimize.limitjointranspose.reductionpercentage=0.1f;
+set hive.optimize.limitjointranspose.reductiontuples=100;
+set hive.explain.user=false;
+set hive.auto.convert.join=false;
+set hive.optimize.dynamic.partition.hashjoin=false;
+
+explain
+select a.*
+from alltypesorc a left outer join src b
+on a.cint = cast(b.key as int) and (a.cint < 100)
+limit 1;
+
+
+set hive.auto.convert.join=true;
+set hive.optimize.dynamic.partition.hashjoin=true;
+set hive.auto.convert.join.noconditionaltask.size=200000;
+set hive.exec.reducers.bytes.per.reducer=200000;
+
+explain
+select a.*
+from alltypesorc a left outer join src b
+on a.cint = cast(b.key as int) and (a.cint < 100)
+limit 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/test/results/clientpositive/join32_lessSize.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join32_lessSize.q.out b/ql/src/test/results/clientpositive/join32_lessSize.q.out
index 5ea4024..8e71710 100644
--- a/ql/src/test/results/clientpositive/join32_lessSize.q.out
+++ b/ql/src/test/results/clientpositive/join32_lessSize.q.out
@@ -1920,22 +1920,22 @@ TOK_QUERY
 
 
 STAGE DEPENDENCIES:
-  Stage-9 is a root stage
-  Stage-7 depends on stages: Stage-9
-  Stage-8 depends on stages: Stage-7
+  Stage-8 is a root stage
   Stage-6 depends on stages: Stage-8
-  Stage-0 depends on stages: Stage-6
-  Stage-3 depends on stages: Stage-0
+  Stage-7 depends on stages: Stage-6
+  Stage-5 depends on stages: Stage-7
+  Stage-0 depends on stages: Stage-5
+  Stage-2 depends on stages: Stage-0
 
 STAGE PLANS:
-  Stage: Stage-9
+  Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_0:$hdt$_0:y 
+        $hdt$_1:$hdt$_1:y 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        $hdt$_0:$hdt$_0:y 
+        $hdt$_1:$hdt$_1:y 
           TableScan
             alias: y
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1950,7 +1950,7 @@ STAGE PLANS:
                   1 _col0 (type: string)
                 Position of Big Table: 1
 
-  Stage: Stage-7
+  Stage: Stage-6
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -2089,12 +2089,12 @@ STAGE PLANS:
               name: default.src1
             name: default.src1
       Truncated Path -> Alias:
-        /src1 [$hdt$_0:$hdt$_1:x]
+        /src1 [$hdt$_1:$hdt$_2:x]
 
-  Stage: Stage-8
+  Stage: Stage-7
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        $hdt$_1:z 
+        $hdt$_0:z 
           Fetch Operator
             limit: -1
             Partition Description:
@@ -2144,7 +2144,7 @@ STAGE PLANS:
                     name: default.srcpart
                   name: default.srcpart
       Alias -> Map Local Operator Tree:
-        $hdt$_1:z 
+        $hdt$_0:z 
           TableScan
             alias: z
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -2159,11 +2159,11 @@ STAGE PLANS:
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 HashTable Sink Operator
                   keys:
-                    0 _col1 (type: string)
-                    1 _col0 (type: string)
-                  Position of Big Table: 0
+                    0 _col0 (type: string)
+                    1 _col1 (type: string)
+                  Position of Big Table: 1
 
-  Stage: Stage-6
+  Stage: Stage-5
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -2172,13 +2172,13 @@ STAGE PLANS:
               condition map:
                    Inner Join 0 to 1
               keys:
-                0 _col1 (type: string)
-                1 _col0 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Position of Big Table: 0
+                0 _col0 (type: string)
+                1 _col1 (type: string)
+              outputColumnNames: _col0, _col3, _col4
+              Position of Big Table: 1
               Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
               Select Operator
-                expressions: _col0 (type: string), _col2 (type: string), _col1 (type: string)
+                expressions: _col3 (type: string), _col0 (type: string), _col4 (type: string)
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
@@ -2313,7 +2313,7 @@ STAGE PLANS:
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: default.dest_j2
 
-  Stage: Stage-3
+  Stage: Stage-2
     Stats-Aggr Operator
 #### A masked pattern was here ####
 

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/test/results/clientpositive/join_merging.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_merging.q.out b/ql/src/test/results/clientpositive/join_merging.q.out
index b42bdc5..9ab9587 100644
--- a/ql/src/test/results/clientpositive/join_merging.q.out
+++ b/ql/src/test/results/clientpositive/join_merging.q.out
@@ -97,12 +97,12 @@ from part p1 left outer join part p2 on p1.p_partkey = p2.p_partkey
               p1.p_size > 10 and p1.p_size > p2.p_size + 10
 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-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
 
 STAGE PLANS:
-  Stage: Stage-1
+  Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan
@@ -153,17 +153,10 @@ STAGE PLANS:
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-  Stage: Stage-2
+  Stage: Stage-1
     Map Reduce
       Map Operator Tree:
           TableScan
-            Reduce Output Operator
-              key expressions: _col2 (type: int)
-              sort order: +
-              Map-reduce partition columns: _col2 (type: int)
-              Statistics: Num rows: 9 Data size: 1112 Basic stats: COMPLETE Column stats: NONE
-              value expressions: _col1 (type: int), _col3 (type: int)
-          TableScan
             alias: p1
             Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
             Select Operator
@@ -175,17 +168,24 @@ STAGE PLANS:
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+          TableScan
+            Reduce Output Operator
+              key expressions: _col2 (type: int)
+              sort order: +
+              Map-reduce partition columns: _col2 (type: int)
+              Statistics: Num rows: 9 Data size: 1112 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: int), _col3 (type: int)
       Reduce Operator Tree:
         Join Operator
           condition map:
-               Right Outer Join0 to 1
+               Left Outer Join0 to 1
           keys:
-            0 _col2 (type: int)
-            1 _col0 (type: int)
-          outputColumnNames: _col1, _col3
+            0 _col0 (type: int)
+            1 _col2 (type: int)
+          outputColumnNames: _col2, _col4
           Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
           Select Operator
-            expressions: _col1 (type: int), _col3 (type: int)
+            expressions: _col2 (type: int), _col4 (type: int)
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
             File Output Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/39bd58b2/ql/src/test/results/clientpositive/limit_join_transpose.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/limit_join_transpose.q.out b/ql/src/test/results/clientpositive/limit_join_transpose.q.out
new file mode 100644
index 0000000..8717154
--- /dev/null
+++ b/ql/src/test/results/clientpositive/limit_join_transpose.q.out
@@ -0,0 +1,869 @@
+PREHOOK: query: explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 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
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0	0	val_0
+PREHOOK: query: explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 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
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0	0	val_0
+PREHOOK: query: explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+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 PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Limit
+                Number of rows: 1
+                Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 10 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
+              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
+              key expressions: _col0 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: string)
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: string)
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 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
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+from src src1 left outer join src src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+238	val_238	238	val_238
+PREHOOK: query: explain
+select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-3 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-3
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Limit
+                Number of rows: 1
+                Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 10 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
+              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
+              key expressions: _col1 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col1 (type: string)
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string)
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col1 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col1 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 _col1 (type: string)
+            1 _col1 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+          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-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col2 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col2 (type: string)
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 _col2 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col4 (type: string), _col5 (type: string), _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+            Statistics: Num rows: 605 Data size: 6427 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
+                Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+                table:
+                    input format: org.apache.hadoop.mapred.TextInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+238	val_238	238	val_238	238	val_238
+PREHOOK: query: explain
+select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-3 depends on stages: Stage-2
+  Stage-4 depends on stages: Stage-3
+  Stage-1 depends on stages: Stage-4
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Limit
+                Number of rows: 1
+                Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: string), _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 10 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
+              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-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col1 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col1 (type: string)
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string)
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col1 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col1 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col0 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Left Outer Join0 to 1
+          keys:
+            0 _col1 (type: string)
+            1 _col1 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 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
+              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-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 10 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
+              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-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+          TableScan
+            Reduce Output Operator
+              key expressions: _col2 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col2 (type: string)
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Right Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col2 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 550 Data size: 5843 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
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+from src src1 right outer join (
+  select *
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+238	val_238	238	val_238	238	val_238
+PREHOOK: query: explain
+select *
+from src src1 right outer join (
+  select src2.key, src2.value
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+order by src2.key
+limit 1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select *
+from src src1 right outer join (
+  select src2.key, src2.value
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+order by src2.key
+limit 1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-5 is a root stage
+  Stage-3 depends on stages: Stage-5
+  Stage-4 depends on stages: Stage-3
+  Stage-1 depends on stages: Stage-4
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-5
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 500 Data size: 5312 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
+              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-3
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+          TableScan
+            Reduce Output Operator
+              key expressions: _col1 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col1 (type: string)
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Right Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col1 (type: string)
+          outputColumnNames: _col1, _col2
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col1 (type: string), _col2 (type: string)
+            outputColumnNames: _col0, _col1
+            Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+            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-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: string)
+              sort order: +
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 550 Data size: 5843 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
+              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-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src1
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                value expressions: _col1 (type: string)
+          TableScan
+            Reduce Output Operator
+              key expressions: _col0 (type: string)
+              sort order: +
+              Map-reduce partition columns: _col0 (type: string)
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col1 (type: string)
+      Reduce Operator Tree:
+        Join Operator
+          condition map:
+               Right Outer Join0 to 1
+          keys:
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+          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
+              key expressions: _col2 (type: string)
+              sort order: +
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+      Reduce Operator Tree:
+        Select Operator
+          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 5843 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
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select *
+from src src1 right outer join (
+  select src2.key, src2.value
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+order by src2.key
+limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+from src src1 right outer join (
+  select src2.key, src2.value
+  from src src2 left outer join src src3
+  on src2.value = src3.value) src2
+on src1.key = src2.key
+order by src2.key
+limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0	0	val_0