You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2020/03/31 17:30:41 UTC

[hive] branch master updated: HIVE-22785: Update/delete/merge statements not optimized through CBO (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)

This is an automated email from the ASF dual-hosted git repository.

jcamacho pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new aa142d1  HIVE-22785: Update/delete/merge statements not optimized through CBO (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)
aa142d1 is described below

commit aa142d1f630b9dd2b412f6719528588b34ef2fe5
Author: Krisztian Kasa <kk...@cloudera.com>
AuthorDate: Tue Mar 31 10:29:48 2020 -0700

    HIVE-22785: Update/delete/merge statements not optimized through CBO (Krisztian Kasa, reviewed by Jesus Camacho Rodriguez)
---
 .../clientpositive/map_join_on_filter.q.out        |    1 +
 .../test/resources/testconfiguration.properties    |    2 +
 .../org/apache/hadoop/hive/ql/QueryProperties.java |    9 +
 .../ql/optimizer/calcite/HiveRelDistribution.java  |   28 +-
 .../ql/optimizer/calcite/HiveRelFactories.java     |   14 +
 .../hive/ql/optimizer/calcite/HiveRelJson.java     |   63 +
 .../hive/ql/optimizer/calcite/HiveRelJsonImpl.java |   10 +
 .../hive/ql/optimizer/calcite/HiveRelOptUtil.java  |   96 +
 .../calcite/reloperators/HiveSortExchange.java     |   66 +-
 .../HiveProjectSortExchangeTransposeRule.java      |  102 +
 .../rules/HiveProjectSortTransposeRule.java        |   74 +-
 .../calcite/rules/HiveRelFieldTrimmer.java         |   48 +
 .../rules/HiveSortLimitPullUpConstantsRule.java    |  170 -
 .../calcite/rules/HiveSortPullUpConstantsRule.java |  209 +
 .../optimizer/calcite/translator/ASTConverter.java |  166 +-
 .../calcite/translator/PlanModifierForASTConv.java |   12 +-
 .../opconventer/HiveSortExchangeVisitor.java       |    8 +-
 .../translator/opconventer/JoinVisitor.java        |    2 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |  500 +-
 .../hive/ql/parse/RewriteSemanticAnalyzer.java     |    2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java     |   27 +-
 .../authorization_view_disable_cbo_1.q             |    2 +-
 ql/src/test/queries/clientpositive/sort.q          |    5 +
 ql/src/test/queries/clientpositive/sort_acid.q     |   35 +
 .../materialized_view_no_cbo_rewrite.q.out         |    2 +-
 .../materialized_view_no_cbo_rewrite_2.q.out       |    2 +-
 .../clientnegative/update_notnull_constraint.q.out |    7 +-
 .../results/clientpositive/acid_view_delete.q.out  |    2 +-
 .../authorization_view_disable_cbo_1.q.out         |    4 +-
 .../test/results/clientpositive/auto_join0.q.out   |  106 +-
 .../test/results/clientpositive/auto_join15.q.out  |   58 +-
 .../test/results/clientpositive/auto_join20.q.out  |  316 +-
 .../test/results/clientpositive/auto_join21.q.out  |  160 +-
 .../test/results/clientpositive/auto_join23.q.out  |   48 +-
 .../test/results/clientpositive/auto_join28.q.out  |  500 +-
 .../test/results/clientpositive/auto_join29.q.out  | 1067 +--
 .../test/results/clientpositive/auto_join31.q.out  |   89 +-
 .../results/clientpositive/cbo_rp_auto_join0.q.out |   92 +-
 .../clientpositive/correlationoptimizer14.q.out    |   70 +-
 .../identity_project_remove_skip.q.out             |   20 +-
 .../test/results/clientpositive/input_part7.q.out  |   42 +-
 ql/src/test/results/clientpositive/join0.q.out     |   16 +-
 ql/src/test/results/clientpositive/join15.q.out    |   54 +-
 ql/src/test/results/clientpositive/join20.q.out    |  208 +-
 ql/src/test/results/clientpositive/join21.q.out    |  115 +-
 ql/src/test/results/clientpositive/join23.q.out    |   54 +-
 ql/src/test/results/clientpositive/join40.q.out    |  208 +-
 .../clientpositive/llap/acid_no_buckets.q.out      |  464 +-
 .../llap/acid_vectorization_original.q.out         |   16 +-
 .../results/clientpositive/llap/auto_join0.q.out   |   30 +-
 .../results/clientpositive/llap/auto_join21.q.out  |  107 +-
 .../results/clientpositive/llap/auto_join29.q.out  |  960 +--
 .../results/clientpositive/llap/auto_join30.q.out  |  538 +-
 .../clientpositive/llap/check_constraint.q.out     |  739 +-
 .../llap/dynamic_semijoin_reduction_3.q.out        |  676 +-
 .../llap/dynpart_sort_optimization_acid.q.out      |   95 +-
 .../llap/enforce_constraint_notnull.q.out          | 1273 +--
 .../clientpositive/llap/explainuser_1.q.out        |   54 +-
 .../llap/identity_project_remove_skip.q.out        |   20 +-
 .../llap/insert_into_default_keyword.q.out         |  429 +-
 .../test/results/clientpositive/llap/join0.q.out   |   14 +-
 .../results/clientpositive/llap/llap_acid.q.out    |    2 +-
 .../clientpositive/llap/llap_acid_fast.q.out       |    2 +-
 .../llap/orc_predicate_pushdown.q.out              |   26 +-
 .../llap/parquet_predicate_pushdown.q.out          |   26 +-
 .../clientpositive/llap/runtime_stats_merge.q.out  |   82 +-
 .../results/clientpositive/llap/semijoin.q.out     |  766 +-
 .../results/clientpositive/llap/sort_acid.q.out    |  149 +
 .../results/clientpositive/llap/sqlmerge.q.out     |  360 +-
 .../clientpositive/llap/sqlmerge_stats.q.out       | 1902 ++---
 .../clientpositive/llap/vector_join30.q.out        | 1644 ++--
 .../llap/vector_leftsemi_mapjoin.q.out             | 8277 +++++++++++---------
 .../llap/vector_mapjoin_complex_values.q.out       |  127 +-
 .../llap/vector_outer_reference_windowed.q.out     |   20 +-
 .../llap/vector_windowing_gby2.q.out               |    2 +-
 .../mapjoin_filter_on_outerjoin.q.out              |  258 +-
 .../clientpositive/mapjoin_test_outer.q.out        |  210 +-
 ql/src/test/results/clientpositive/no_hooks.q.out  |   54 +-
 .../results/clientpositive/parallel_join0.q.out    |   16 +-
 ql/src/test/results/clientpositive/pcs.q.out       |   64 +-
 .../clientpositive/perf/tez/cbo_query51.q.out      |   42 +-
 .../perf/tez/constraints/cbo_query51.q.out         |   42 +-
 .../perf/tez/constraints/query51.q.out             |   96 +-
 .../perf/tez/constraints/query70.q.out             |  104 +-
 .../results/clientpositive/perf/tez/query51.q.out  |   96 +-
 .../results/clientpositive/perf/tez/query70.q.out  |  104 +-
 ql/src/test/results/clientpositive/ppd_join4.q.out |   32 +-
 ql/src/test/results/clientpositive/sort.q.out      |  559 ++
 .../results/clientpositive/spark/auto_join0.q.out  |   34 +-
 .../results/clientpositive/spark/auto_join15.q.out |   58 +-
 .../results/clientpositive/spark/auto_join20.q.out |  228 +-
 .../results/clientpositive/spark/auto_join21.q.out |   92 +-
 .../results/clientpositive/spark/auto_join23.q.out |   48 +-
 .../results/clientpositive/spark/auto_join28.q.out |  395 +-
 .../results/clientpositive/spark/auto_join29.q.out |  936 ++-
 .../results/clientpositive/spark/auto_join30.q.out |  434 +-
 .../results/clientpositive/spark/auto_join31.q.out |  110 +-
 .../clientpositive/spark/dynamic_rdd_cache.q.out   |   90 +-
 .../spark/identity_project_remove_skip.q.out       |   16 +-
 .../test/results/clientpositive/spark/join0.q.out  |   14 +-
 .../test/results/clientpositive/spark/join15.q.out |   50 +-
 .../test/results/clientpositive/spark/join20.q.out |  164 +-
 .../test/results/clientpositive/spark/join21.q.out |   86 +-
 .../test/results/clientpositive/spark/join23.q.out |   50 +-
 .../test/results/clientpositive/spark/join40.q.out |  164 +-
 .../spark/mapjoin_filter_on_outerjoin.q.out        |  237 +-
 .../clientpositive/spark/mapjoin_test_outer.q.out  |  197 +-
 .../clientpositive/spark/parallel_join0.q.out      |   14 +-
 .../results/clientpositive/spark/ppd_join4.q.out   |   30 +-
 .../results/clientpositive/spark/semijoin.q.out    | 1045 +--
 .../test/results/clientpositive/spark/sort.q.out   |  566 ++
 .../clientpositive/spark/spark_explainuser_1.q.out |   34 +-
 .../results/clientpositive/spark/union_ppr.q.out   |   38 +-
 .../tez/acid_vectorization_original_tez.q.out      |   16 +-
 ql/src/test/results/clientpositive/union_ppr.q.out |   42 +-
 115 files changed, 17638 insertions(+), 12918 deletions(-)

diff --git a/itests/hive-blobstore/src/test/results/clientpositive/map_join_on_filter.q.out b/itests/hive-blobstore/src/test/results/clientpositive/map_join_on_filter.q.out
index 653faab..087fa94 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/map_join_on_filter.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/map_join_on_filter.q.out
@@ -28,6 +28,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/smbbucket_1.txt' INTO
 POSTHOOK: type: LOAD
 #### A masked pattern was here ####
 POSTHOOK: Output: default@src_a_data
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT /*+ MAPJOIN(src1, src2) */ *
 FROM src_a_data src1
 RIGHT OUTER JOIN src_a_data src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10)
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 3510016..cfe83d6 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -15,6 +15,7 @@ disabled.query.files=cbo_rp_subq_in.q,\
   sample2.q,\
   sample4.q,\
   root_dir_external_table.q,\
+  sort_acid.q,\
   input31.q
 
 
@@ -210,6 +211,7 @@ minillaplocal.shared.query.files=alter_merge_2_orc.q,\
   selectDistinctStar.q,\
   select_dummy_source.q,\
   skewjoin.q,\
+  sort_acid.q,\
   stats_noscan_1.q,\
   stats_only_null.q,\
   subquery_exists.q,\
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryProperties.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryProperties.java
index 9c61b31..d9d338c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/QueryProperties.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryProperties.java
@@ -45,6 +45,7 @@ public class QueryProperties {
   boolean hasOrderBy = false;
   boolean hasOuterOrderBy = false;
   boolean hasSortBy = false;
+  boolean hasLimit = false;
   boolean hasJoinFollowedByGroupBy = false;
   boolean hasPTF = false;
   boolean hasWindowing = false;
@@ -184,6 +185,14 @@ public class QueryProperties {
     this.hasSortBy = hasSortBy;
   }
 
+  public void setHasLimit(boolean hasLimit) {
+    this.hasLimit = hasLimit;
+  }
+
+  public boolean hasLimit() {
+    return hasLimit;
+  }
+
   public boolean hasJoinFollowedByGroupBy() {
     return hasJoinFollowedByGroupBy;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelDistribution.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelDistribution.java
index e5f4c84..2caaba0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelDistribution.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelDistribution.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.calcite.plan.RelMultipleTrait;
@@ -25,14 +26,23 @@ import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributionTraitDef;
+import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.util.mapping.Mappings.TargetMapping;
 
 import com.google.common.collect.Ordering;
 
 public class HiveRelDistribution implements RelDistribution {
 
-  private static final Ordering<Iterable<Integer>> ORDERING =
-      Ordering.<Integer>natural().lexicographical();
+  private static final Ordering<Iterable<Integer>> ORDERING = Ordering.<Integer>natural().lexicographical();
+
+  public static HiveRelDistribution from(
+          List<RelFieldCollation> fieldCollations, RelDistribution.Type distributionType) {
+    List<Integer> newDistributionKeys = new ArrayList<>(fieldCollations.size());
+    for (RelFieldCollation fieldCollation : fieldCollations) {
+      newDistributionKeys.add(fieldCollation.getFieldIndex());
+    }
+    return new HiveRelDistribution(distributionType, newDistributionKeys);
+  }
 
   List<Integer> keys;
   RelDistribution.Type type;
@@ -70,7 +80,11 @@ public class HiveRelDistribution implements RelDistribution {
     if (keys.isEmpty()) {
       return this;
     }
-    return new HiveRelDistribution(type, keys);
+    List<Integer> newKeys = new ArrayList<>(keys.size());
+    for (Integer key : keys) {
+      newKeys.add(mapping.getTargetOpt(key));
+    }
+    return new HiveRelDistribution(type, newKeys);
   }
 
   @Override
@@ -100,4 +114,12 @@ public class HiveRelDistribution implements RelDistribution {
     return type.compareTo(distribution.getType());
   }
 
+  @Override
+  public String toString() {
+    if (keys.isEmpty()) {
+      return type.shortName;
+    } else {
+      return type.shortName + keys;
+    }
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
index 04b3888..6be826e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
@@ -25,10 +25,12 @@ import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.RelFactories.AggregateFactory;
 import org.apache.calcite.rel.core.RelFactories.FilterFactory;
 import org.apache.calcite.rel.core.RelFactories.JoinFactory;
@@ -50,6 +52,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIntersect;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
 import com.google.common.collect.ImmutableList;
@@ -71,6 +74,9 @@ public class HiveRelFactories {
   public static final SortFactory HIVE_SORT_FACTORY =
           new HiveSortFactoryImpl();
 
+  public static final RelFactories.SortExchangeFactory HIVE_SORT_EXCHANGE_FACTORY =
+          new HiveSortExchangeFactoryImpl();
+
   public static final AggregateFactory HIVE_AGGREGATE_FACTORY =
           new HiveAggregateFactoryImpl();
 
@@ -85,6 +91,7 @@ public class HiveRelFactories {
               HIVE_JOIN_FACTORY,
               HIVE_SEMI_JOIN_FACTORY,
               HIVE_SORT_FACTORY,
+              HIVE_SORT_EXCHANGE_FACTORY,
               HIVE_AGGREGATE_FACTORY,
               HIVE_SET_OP_FACTORY));
 
@@ -195,6 +202,13 @@ public class HiveRelFactories {
     }
   }
 
+  private static class HiveSortExchangeFactoryImpl implements RelFactories.SortExchangeFactory {
+    @Override
+    public RelNode createSortExchange(RelNode input, RelDistribution distribution, RelCollation collation) {
+      return HiveSortExchange.create(input, distribution, collation);
+    }
+  }
+
   private static class HiveAggregateFactoryImpl implements AggregateFactory {
     @Override
     public RelNode createAggregate(RelNode child,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelJson.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelJson.java
new file mode 100644
index 0000000..31c9a9c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelJson.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.externalize.RelJson;
+import org.apache.calcite.util.JsonBuilder;
+
+/**
+ * Hive extension of RelJson.
+ * Implement json serialization of types which are not support by Calcite 1.21.0.
+ * This class can be removed when Calcite is upgraded to 1.23.0
+ */
+public class HiveRelJson extends RelJson {
+  private final JsonBuilder jsonBuilder;
+
+  public HiveRelJson(JsonBuilder jsonBuilder) {
+    super(jsonBuilder);
+    this.jsonBuilder = jsonBuilder;
+  }
+
+  @Override
+  public Object toJson(Object value) {
+    if (value instanceof RelDistribution) {
+      return toJson((RelDistribution) value);
+    }
+    return super.toJson(value);
+  }
+
+  // Upgrade to Calcite 1.23.0 to remove this method
+  private Object toJson(RelDistribution relDistribution) {
+    final Map<String, Object> map = jsonBuilder.map();
+    map.put("type", relDistribution.getType().name());
+
+    if (!relDistribution.getKeys().isEmpty()) {
+      List<Object> keys = new ArrayList<>(relDistribution.getKeys().size());
+      for (Integer key : relDistribution.getKeys()) {
+        keys.add(toJson(key));
+      }
+      map.put("keys", keys);
+    }
+    return map;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelJsonImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelJsonImpl.java
index 0d45eb0..65877e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelJsonImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelJsonImpl.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite;
 
+import java.lang.reflect.Field;
 import java.util.List;
 import java.util.Map;
 import org.apache.calcite.rel.RelNode;
@@ -40,6 +41,15 @@ public class HiveRelJsonImpl extends RelJsonWriter {
 
   public HiveRelJsonImpl() {
     super();
+
+    // Upgrade to Calcite 1.23.0 to remove this
+    try {
+      final Field fieldRelJson = RelJsonWriter.class.getDeclaredField("relJson");
+      fieldRelJson.setAccessible(true);
+      fieldRelJson.set(this, new HiveRelJson(jsonBuilder));
+    } catch (IllegalAccessException | NoSuchFieldException e) {
+      throw new RuntimeException(e);
+    }
   }
 
   //~ Methods ------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
index e647b88..1e2c1e2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
@@ -35,6 +35,9 @@ import java.util.Set;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelReferentialConstraint;
 import org.apache.calcite.rel.core.Aggregate;
@@ -52,6 +55,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCallBinding;
 import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
@@ -61,10 +65,13 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.mapping.Mappings;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
@@ -1048,4 +1055,93 @@ public class HiveRelOptUtil extends RelOptUtil {
     return planWriter.asString();
   }
 
+  /**
+   * Map Sort and SortExchange keys to the specified Project columns.
+   * @param project the Project
+   * @param sortCollation current collation in Sort
+   * @param cluster RelOptCluster
+   * @return new collation should be used in the Sort
+   */
+  public static List<RelFieldCollation> getNewRelFieldCollations(
+          HiveProject project, RelCollation sortCollation, RelOptCluster cluster) {
+    // Determine mapping between project input and output fields.
+    // In Hive, Sort is always based on RexInputRef
+    // HiveSort*PullUpConstantsRule should remove constants (RexLiteral)
+    // We only need to check if project can contain all the positions that sortCollation needs.
+    final Mappings.TargetMapping map =
+            RelOptUtil.permutationIgnoreCast(
+                    project.getProjects(), project.getInput().getRowType()).inverse();
+    Set<Integer> needed = new HashSet<>();
+    for (RelFieldCollation fc : sortCollation.getFieldCollations()) {
+      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
+        final RexCall cast = (RexCall) node;
+        final RexCallBinding binding =
+                RexCallBinding.create(cluster.getTypeFactory(), cast,
+                        ImmutableList.of(RexUtil.apply(map, sortCollation)));
+        if (cast.getOperator().getMonotonicity(binding) == SqlMonotonicity.NOT_MONOTONIC) {
+          return null;
+        }
+      }
+    }
+    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) {
+          int parentPos = positions.iterator().next();
+          if(needed.contains(parentPos)){
+            m.put(parentPos, projPos);
+            needed.remove(parentPos);
+          }
+        }
+      }
+    }
+    if(!needed.isEmpty()){
+      return null;
+    }
+
+    List<RelFieldCollation> fieldCollations = new ArrayList<>();
+    for (RelFieldCollation fc : sortCollation.getFieldCollations()) {
+      fieldCollations.add(new RelFieldCollation(m.get(fc.getFieldIndex()), fc.direction, fc.nullDirection));
+    }
+    return fieldCollations;
+  }
+
+  /**
+   * Map Exchange distribution keys to the specified Project columns.
+   * @param project the Project
+   * @param distribution current distribution in Exchange
+   * @return new distribution should be used in the Exchange
+   */
+  public static List<Integer> getNewRelDistributionKeys(
+          HiveProject project, RelDistribution distribution) {
+    Set<Integer> needed = new HashSet<>(distribution.getKeys());
+    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) {
+          int parentPos = positions.iterator().next();
+          if(needed.contains(parentPos)){
+            m.put(parentPos, projPos);
+            needed.remove(parentPos);
+          }
+        }
+      }
+    }
+    if(!needed.isEmpty()){
+      return null;
+    }
+
+    List<Integer> distributionKeys = new ArrayList<>();
+    for (Integer keyIndex : distribution.getKeys()) {
+      distributionKeys.add(m.get(keyIndex));
+    }
+    return distributionKeys;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortExchange.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortExchange.java
index 880cae7..8c46e44 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortExchange.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveSortExchange.java
@@ -21,24 +21,31 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributionTraitDef;
+import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.SortExchange;
 import org.apache.calcite.rex.RexNode;
+import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 
 import com.google.common.collect.ImmutableList;
 
-public class HiveSortExchange extends SortExchange {
-  private ImmutableList<RexNode> joinKeys;
-  private ExprNodeDesc[] joinExpressions;
+/**
+ * Hive extension of calcite SortExchange.
+ * Add support of keys used when sorting or joining.
+ */
+public final class HiveSortExchange extends SortExchange implements HiveRelNode {
+  private final ImmutableList<RexNode> keys;
+  private ExprNodeDesc[] keyExpressions;
 
   private HiveSortExchange(RelOptCluster cluster, RelTraitSet traitSet,
-      RelNode input, RelDistribution distribution, RelCollation collation, ImmutableList<RexNode> joinKeys) {
+      RelNode input, RelDistribution distribution, RelCollation collation, ImmutableList<RexNode> keys) {
     super(cluster, traitSet, input, distribution, collation);
-    this.joinKeys = new ImmutableList.Builder<RexNode>().addAll(joinKeys).build();
+    this.keys = new ImmutableList.Builder<RexNode>().addAll(keys).build();
   }
 
   /**
@@ -47,38 +54,59 @@ public class HiveSortExchange extends SortExchange {
    * @param input     Input relational expression
    * @param distribution Distribution specification
    * @param collation Collation specification
-   * @param joinKeys Join Keys specification
+   * @param keys Keys specification
    */
   public static HiveSortExchange create(RelNode input,
-      RelDistribution distribution, RelCollation collation, ImmutableList<RexNode> joinKeys) {
+      RelDistribution distribution, RelCollation collation, ImmutableList<RexNode> keys) {
     RelOptCluster cluster = input.getCluster();
     distribution = RelDistributionTraitDef.INSTANCE.canonize(distribution);
     collation = RelCollationTraitDef.INSTANCE.canonize(collation);
-    RelTraitSet traitSet = RelTraitSet.createEmpty().plus(distribution).plus(collation);
-    return new HiveSortExchange(cluster, traitSet, input, distribution, collation, joinKeys);
+    RelTraitSet traitSet = getTraitSet(collation, cluster);
+    return new HiveSortExchange(cluster, traitSet, input, distribution, collation, keys);
+  }
+
+  private static RelTraitSet getTraitSet(RelCollation collation, RelOptCluster cluster) {
+    // add distribution
+    return TraitsUtil.getDefaultTraitSet(cluster).replace(collation);
+  }
+
+  public static HiveSortExchange create(RelNode input,
+      RelDistribution distribution, RelCollation collation) {
+    RelOptCluster cluster = input.getCluster();
+    distribution = RelDistributionTraitDef.INSTANCE.canonize(distribution);
+    collation = RelCollationTraitDef.INSTANCE.canonize(collation);
+    RelTraitSet traitSet = getTraitSet(collation, cluster);
+    RelCollation canonizedCollation = traitSet.canonize(RelCollationImpl.of(collation.getFieldCollations()));
+
+    ImmutableList.Builder<RexNode> builder = ImmutableList.builder();
+    for (RelFieldCollation relFieldCollation : canonizedCollation.getFieldCollations()) {
+      int index = relFieldCollation.getFieldIndex();
+      builder.add(cluster.getRexBuilder().makeInputRef(input, index));
+    }
+
+    return new HiveSortExchange(cluster, traitSet, input, distribution, collation, builder.build());
   }
 
   @Override
   public SortExchange copy(RelTraitSet traitSet, RelNode newInput, RelDistribution newDistribution,
           RelCollation newCollation) {
     return new HiveSortExchange(getCluster(), traitSet, newInput,
-            newDistribution, newCollation, joinKeys);
+            newDistribution, newCollation, keys);
   }
 
-  public ImmutableList<RexNode> getJoinKeys() {
-    return joinKeys;
+  public ImmutableList<RexNode> getKeys() {
+    return keys;
   }
 
-  public void setJoinKeys(ImmutableList<RexNode> joinKeys) {
-    this.joinKeys = joinKeys;
+  public ExprNodeDesc[] getKeyExpressions() {
+    return keyExpressions;
   }
 
-  public ExprNodeDesc[] getJoinExpressions() {
-    return joinExpressions;
+  public void setKeyExpressions(ExprNodeDesc[] keyExpressions) {
+    this.keyExpressions = keyExpressions;
   }
 
-  public void setJoinExpressions(ExprNodeDesc[] joinExpressions) {
-    this.joinExpressions = joinExpressions;
+  @Override
+  public void implement(Implementor implementor) {
   }
-
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortExchangeTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortExchangeTransposeRule.java
new file mode 100644
index 0000000..06cbe3d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortExchangeTransposeRule.java
@@ -0,0 +1,102 @@
+/*
+ * 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 static org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil.getNewRelDistributionKeys;
+import static org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil.getNewRelFieldCollations;
+
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.SortExchange;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelDistribution;
+import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Push down Projection above SortExchange.
+ * HiveProject
+ *   HiveSortExchange
+ *     ...
+ *
+ * =>
+ *
+ * HiveSortExchange
+ *   HiveProject
+ *     ...
+ */
+public final class HiveProjectSortExchangeTransposeRule extends RelOptRule {
+  public static final HiveProjectSortExchangeTransposeRule INSTANCE = new HiveProjectSortExchangeTransposeRule();
+
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Creates a HiveProjectSortTransposeRule.
+   */
+  private HiveProjectSortExchangeTransposeRule() {
+    super(
+        operand(
+            HiveProject.class,
+            operand(HiveSortExchange.class, any())));
+  }
+
+  protected HiveProjectSortExchangeTransposeRule(RelOptRuleOperand operand) {
+    super(operand);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  // implement RelOptRule
+  public void onMatch(RelOptRuleCall call) {
+    final HiveProject project = call.rel(0);
+    final HiveSortExchange sortExchange = call.rel(1);
+    final RelOptCluster cluster = project.getCluster();
+
+    List<RelFieldCollation> fieldCollations = getNewRelFieldCollations(project, sortExchange.getCollation(), cluster);
+    if (fieldCollations == null) {
+      return;
+    }
+
+    RelTraitSet newTraitSet = TraitsUtil.getDefaultTraitSet(sortExchange.getCluster());
+    RelCollation newCollation = newTraitSet.canonize(RelCollationImpl.of(fieldCollations));
+    newTraitSet = newTraitSet.replace(newCollation);
+    List<Integer> newDistributionKeys = getNewRelDistributionKeys(project, sortExchange.getDistribution());
+
+    // New operators
+    final RelNode newProject = project.copy(sortExchange.getInput().getTraitSet(),
+        ImmutableList.of(sortExchange.getInput()));
+    final SortExchange newSort = sortExchange.copy(
+        newTraitSet,
+        newProject,
+        new HiveRelDistribution(sortExchange.getDistribution().getType(), newDistributionKeys),
+        newCollation);
+
+    call.transformTo(newSort);
+  }
+}
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 871c411..9ac9553 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
@@ -17,32 +17,19 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
+import static org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil.getNewRelFieldCollations;
+
 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.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;
@@ -51,8 +38,7 @@ import com.google.common.collect.ImmutableList;
 
 public class HiveProjectSortTransposeRule extends RelOptRule {
 
-  public static final HiveProjectSortTransposeRule INSTANCE =
-      new HiveProjectSortTransposeRule();
+  public static final HiveProjectSortTransposeRule INSTANCE = new HiveProjectSortTransposeRule();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -77,64 +63,20 @@ public class HiveProjectSortTransposeRule extends RelOptRule {
     final HiveProject project = call.rel(0);
     final HiveSortLimit sort = call.rel(1);
     final RelOptCluster cluster = project.getCluster();
-
-    // 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()) {
-      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
-        final RexCall cast = (RexCall) node;
-        final RexCallBinding binding =
-            RexCallBinding.create(cluster.getTypeFactory(), cast,
-                ImmutableList.of(RexUtil.apply(map, sort.getCollation())));
-        if (cast.getOperator().getMonotonicity(binding) == SqlMonotonicity.NOT_MONOTONIC) {
-          return;
-        }
-      }
-    }
-    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()){
+    List<RelFieldCollation> fieldCollations = getNewRelFieldCollations(project, sort.getCollation(), cluster);
+    if (fieldCollations == null) {
       return;
     }
-    
-    List<RelFieldCollation> fieldCollations = new ArrayList<>();
-    for (RelFieldCollation fc : sort.getCollation().getFieldCollations()) {
-      fieldCollations.add(new RelFieldCollation(m.get(fc.getFieldIndex()), fc.direction,
-          fc.nullDirection));
-    }
 
     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()));
+        ImmutableList.of(sort.getInput()));
     final HiveSortLimit newSort = sort.copy(newProject.getTraitSet(),
-            newProject, newCollation, sort.offset, sort.fetch);
+        newProject, newCollation, sort.offset, sort.fetch);
 
     call.transformTo(newSort);
   }
-
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
index 53d68e8..19ce3ea 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
@@ -31,6 +31,9 @@ import org.apache.calcite.adapter.druid.DruidQuery;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
@@ -67,6 +70,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableFunctionScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.parse.ColumnAccessInfo;
@@ -877,4 +881,48 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
       LOG.warn("Error initializing field trimmer instance", t);
     }
   }
+
+  public TrimResult trimFields(
+          HiveSortExchange exchange,
+          ImmutableBitSet fieldsUsed,
+          Set<RelDataTypeField> extraFields) {
+    final RelDataType rowType = exchange.getRowType();
+    final int fieldCount = rowType.getFieldCount();
+    final RelCollation collation = exchange.getCollation();
+    final RelDistribution distribution = exchange.getDistribution();
+    final RelNode input = exchange.getInput();
+
+    // We use the fields used by the consumer, plus any fields used as exchange
+    // keys.
+    final ImmutableBitSet.Builder inputFieldsUsed = fieldsUsed.rebuild();
+    for (RelFieldCollation field : collation.getFieldCollations()) {
+      inputFieldsUsed.set(field.getFieldIndex());
+    }
+    for (int keyIndex : distribution.getKeys()) {
+      inputFieldsUsed.set(keyIndex);
+    }
+
+    // Create input with trimmed columns.
+    final Set<RelDataTypeField> inputExtraFields = Collections.emptySet();
+    TrimResult trimResult =
+            trimChild(exchange, input, inputFieldsUsed.build(), inputExtraFields);
+    RelNode newInput = trimResult.left;
+    final Mapping inputMapping = trimResult.right;
+
+    // If the input is unchanged, and we need to project all columns,
+    // there's nothing we can do.
+    if (newInput == input
+            && inputMapping.isIdentity()
+            && fieldsUsed.cardinality() == fieldCount) {
+      return result(exchange, Mappings.createIdentity(fieldCount));
+    }
+
+    final RelBuilder relBuilder = REL_BUILDER.get();
+    relBuilder.push(newInput);
+    RelCollation newCollation = RexUtil.apply(inputMapping, collation);
+    RelDistribution newDistribution = distribution.apply(inputMapping);
+    relBuilder.sortExchange(newDistribution, newCollation);
+
+    return result(relBuilder.build(), inputMapping);
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
deleted file mode 100644
index e51b2b6..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortLimitPullUpConstantsRule.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.calcite.plan.RelOptPredicateList;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.hep.HepRelVertex;
-import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.mapping.Mappings;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.ImmutableList;
-
-/**
- * Planner rule that pulls up constant keys through a SortLimit operator.
- * 
- * This rule is only applied on SortLimit operators that are not the root
- * of the plan tree. This is done because the interaction of this rule
- * with the AST conversion may cause some optimizations to not kick in
- * e.g. SimpleFetchOptimizer. Nevertheless, this will not have any
- * performance impact in the resulting plans.
- */
-public class HiveSortLimitPullUpConstantsRule extends RelOptRule {
-
-  protected static final Logger LOG = LoggerFactory.getLogger(HiveSortLimitPullUpConstantsRule.class);
-
-
-  public static final HiveSortLimitPullUpConstantsRule INSTANCE =
-          new HiveSortLimitPullUpConstantsRule(HiveSortLimit.class,
-                  HiveRelFactories.HIVE_BUILDER);
-
-  private HiveSortLimitPullUpConstantsRule(Class<? extends Sort> sortClass,
-      RelBuilderFactory relBuilderFactory) {
-    super(operand(RelNode.class, unordered(operand(sortClass, any()))), relBuilderFactory, null);
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final RelNode parent = call.rel(0);
-    final Sort sort = call.rel(1);
-
-    final int count = sort.getInput().getRowType().getFieldCount();
-    if (count == 1) {
-      // No room for optimization since we cannot convert to an empty
-      // Project operator.
-      return;
-    }
-
-    final RexBuilder rexBuilder = sort.getCluster().getRexBuilder();
-    final RelMetadataQuery mq = call.getMetadataQuery();
-    final RelOptPredicateList predicates = mq.getPulledUpPredicates(sort.getInput());
-    if (predicates == null) {
-      return;
-    }
-
-    Map<RexNode, RexNode> conditionsExtracted = HiveReduceExpressionsRule.predicateConstants(
-            RexNode.class, rexBuilder, predicates);
-    Map<RexNode, RexNode> constants = new HashMap<>();
-    for (int i = 0; i < count ; i++) {
-      RexNode expr = rexBuilder.makeInputRef(sort.getInput(), i);
-      if (conditionsExtracted.containsKey(expr)) {
-        constants.put(expr, conditionsExtracted.get(expr));
-      }
-    }
-
-    // None of the expressions are constant. Nothing to do.
-    if (constants.isEmpty()) {
-      return;
-    }
-
-    if (count == constants.size()) {
-      // At least a single item in project is required.
-      constants.remove(constants.keySet().iterator().next());
-    }
-
-    // Create expressions for Project operators before and after the Sort
-    List<RelDataTypeField> fields = sort.getInput().getRowType().getFieldList();
-    List<Pair<RexNode, String>> newChildExprs = new ArrayList<>();
-    List<RexNode> topChildExprs = new ArrayList<>();
-    List<String> topChildExprsFields = new ArrayList<>();
-    for (int i = 0; i < count ; i++) {
-      RexNode expr = rexBuilder.makeInputRef(sort.getInput(), i);
-      RelDataTypeField field = fields.get(i);
-      if (constants.containsKey(expr)) {
-        topChildExprs.add(constants.get(expr));
-        topChildExprsFields.add(field.getName());
-      } else {
-        newChildExprs.add(Pair.<RexNode,String>of(expr, field.getName()));
-        topChildExprs.add(expr);
-        topChildExprsFields.add(field.getName());
-      }
-    }
-
-    // Update field collations
-    final Mappings.TargetMapping mapping =
-            RelOptUtil.permutation(Pair.left(newChildExprs), sort.getInput().getRowType()).inverse();
-    List<RelFieldCollation> fieldCollations = new ArrayList<>();
-    for (RelFieldCollation fc : sort.getCollation().getFieldCollations()) {
-      final int target = mapping.getTargetOpt(fc.getFieldIndex());
-      if (target < 0) {
-        // It is a constant, we can ignore it
-        continue;
-      }
-      fieldCollations.add(fc.copy(target));
-    }
-
-    // Update top Project positions
-    topChildExprs = ImmutableList.copyOf(RexUtil.apply(mapping, topChildExprs));
-
-    // Create new Project-Sort-Project sequence
-    final RelBuilder relBuilder = call.builder();
-    relBuilder.push(sort.getInput());
-    relBuilder.project(Pair.left(newChildExprs), Pair.right(newChildExprs));
-    final ImmutableList<RexNode> sortFields =
-            relBuilder.fields(RelCollations.of(fieldCollations));
-    relBuilder.sortLimit(sort.offset == null ? -1 : RexLiteral.intValue(sort.offset),
-            sort.fetch == null ? -1 : RexLiteral.intValue(sort.fetch), sortFields);
-    // Create top Project fixing nullability of fields
-    relBuilder.project(topChildExprs, topChildExprsFields);
-    relBuilder.convert(sort.getRowType(), false);
-
-    List<RelNode> inputs = new ArrayList<>();
-    for (RelNode child : parent.getInputs()) {
-      if (!((HepRelVertex) child).getCurrentRel().equals(sort)) {
-        inputs.add(child);
-      } else {
-        inputs.add(relBuilder.build());
-      }
-    }
-    call.transformTo(parent.copy(parent.getTraitSet(), inputs));
-  }
-
-}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortPullUpConstantsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortPullUpConstantsRule.java
new file mode 100644
index 0000000..5765ddf
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortPullUpConstantsRule.java
@@ -0,0 +1,209 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Planner rule that pulls up constant keys through a SortLimit or SortExchange operator.
+ *
+ * This rule is only applied on SortLimit operators that are not the root
+ * of the plan tree. This is done because the interaction of this rule
+ * with the AST conversion may cause some optimizations to not kick in
+ * e.g. SimpleFetchOptimizer. Nevertheless, this will not have any
+ * performance impact in the resulting plans.
+ */
+public final class HiveSortPullUpConstantsRule {
+
+  private HiveSortPullUpConstantsRule() {}
+
+  public static final HiveSortPullUpConstantsRuleBase<HiveSortLimit> SORT_LIMIT_INSTANCE =
+      new HiveSortLimitPullUpConstantsRule();
+
+  private static final class HiveSortLimitPullUpConstantsRule
+      extends HiveSortPullUpConstantsRuleBase<HiveSortLimit> {
+
+    protected HiveSortLimitPullUpConstantsRule() {
+      super(HiveSortLimit.class);
+    }
+
+    @Override
+    protected void buildSort(RelBuilder relBuilder, HiveSortLimit sortNode, Mappings.TargetMapping mapping) {
+      List<RelFieldCollation> fieldCollations = applyToFieldCollations(sortNode.getCollation(), mapping);
+      final ImmutableList<RexNode> sortFields =
+          relBuilder.fields(RelCollations.of(fieldCollations));
+      relBuilder.sortLimit(sortNode.offset == null ? -1 : RexLiteral.intValue(sortNode.offset),
+          sortNode.fetch == null ? -1 : RexLiteral.intValue(sortNode.fetch), sortFields);
+    }
+  }
+
+  public static final HiveSortExchangePullUpConstantsRule SORT_EXCHANGE_INSTANCE =
+      new HiveSortExchangePullUpConstantsRule();
+
+  private static final class HiveSortExchangePullUpConstantsRule
+      extends HiveSortPullUpConstantsRuleBase<HiveSortExchange> {
+
+    protected HiveSortExchangePullUpConstantsRule() {
+      super(HiveSortExchange.class);
+    }
+
+    @Override
+    protected void buildSort(RelBuilder relBuilder, HiveSortExchange sortNode, Mappings.TargetMapping mapping) {
+      List<RelFieldCollation> fieldCollations = applyToFieldCollations(sortNode.getCollation(), mapping);
+      RelDistribution distribution = sortNode.getDistribution().apply(mapping);
+      relBuilder.sortExchange(distribution, RelCollations.of(fieldCollations));
+    }
+  }
+
+
+  private abstract static class HiveSortPullUpConstantsRuleBase<T extends SingleRel> extends RelOptRule {
+
+    protected HiveSortPullUpConstantsRuleBase(Class<T> sortClass) {
+      super(operand(RelNode.class, unordered(operand(sortClass, any()))), HiveRelFactories.HIVE_BUILDER, null);
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      final RelNode parent = call.rel(0);
+      final T sortNode = call.rel(1);
+
+      final int count = sortNode.getInput().getRowType().getFieldCount();
+      if (count == 1) {
+        // No room for optimization since we cannot convert to an empty
+        // Project operator.
+        return;
+      }
+
+      final RexBuilder rexBuilder = sortNode.getCluster().getRexBuilder();
+      final RelMetadataQuery mq = call.getMetadataQuery();
+      final RelOptPredicateList predicates = mq.getPulledUpPredicates(sortNode.getInput());
+      if (predicates == null) {
+        return;
+      }
+
+      Map<RexNode, RexNode> conditionsExtracted = HiveReduceExpressionsRule.predicateConstants(
+          RexNode.class, rexBuilder, predicates);
+      Map<RexNode, RexNode> constants = new HashMap<>();
+      for (int i = 0; i < count; i++) {
+        RexNode expr = rexBuilder.makeInputRef(sortNode.getInput(), i);
+        if (conditionsExtracted.containsKey(expr)) {
+          constants.put(expr, conditionsExtracted.get(expr));
+        }
+      }
+
+      // None of the expressions are constant. Nothing to do.
+      if (constants.isEmpty()) {
+        return;
+      }
+
+      if (count == constants.size()) {
+        // At least a single item in project is required.
+        constants.remove(constants.keySet().iterator().next());
+      }
+
+      // Create expressions for Project operators before and after the Sort
+      List<RelDataTypeField> fields = sortNode.getInput().getRowType().getFieldList();
+      List<Pair<RexNode, String>> newChildExprs = new ArrayList<>();
+      List<RexNode> topChildExprs = new ArrayList<>();
+      List<String> topChildExprsFields = new ArrayList<>();
+      for (int i = 0; i < count; i++) {
+        RexNode expr = rexBuilder.makeInputRef(sortNode.getInput(), i);
+        RelDataTypeField field = fields.get(i);
+        if (constants.containsKey(expr)) {
+          topChildExprs.add(constants.get(expr));
+          topChildExprsFields.add(field.getName());
+        } else {
+          newChildExprs.add(Pair.<RexNode, String>of(expr, field.getName()));
+          topChildExprs.add(expr);
+          topChildExprsFields.add(field.getName());
+        }
+      }
+
+      // Update field collations
+      final Mappings.TargetMapping mapping =
+          RelOptUtil.permutation(Pair.left(newChildExprs), sortNode.getInput().getRowType()).inverse();
+
+      // Update top Project positions
+      topChildExprs = ImmutableList.copyOf(RexUtil.apply(mapping, topChildExprs));
+
+      // Create new Project-Sort-Project sequence
+      final RelBuilder relBuilder = call.builder();
+      relBuilder.push(sortNode.getInput());
+      relBuilder.project(Pair.left(newChildExprs), Pair.right(newChildExprs));
+      buildSort(relBuilder, sortNode, mapping);
+      // Create top Project fixing nullability of fields
+      relBuilder.project(topChildExprs, topChildExprsFields);
+      relBuilder.convert(sortNode.getRowType(), false);
+
+      List<RelNode> inputs = new ArrayList<>();
+      for (RelNode child : parent.getInputs()) {
+        if (!((HepRelVertex) child).getCurrentRel().equals(sortNode)) {
+          inputs.add(child);
+        } else {
+          inputs.add(relBuilder.build());
+        }
+      }
+      call.transformTo(parent.copy(parent.getTraitSet(), inputs));
+    }
+
+    protected List<RelFieldCollation> applyToFieldCollations(
+        RelCollation relCollation, Mappings.TargetMapping mapping) {
+      List<RelFieldCollation> fieldCollations = new ArrayList<>();
+      for (RelFieldCollation fc : relCollation.getFieldCollations()) {
+        final int target = mapping.getTargetOpt(fc.getFieldIndex());
+        if (target < 0) {
+          // It is a constant, we can ignore it
+          continue;
+        }
+        fieldCollations.add(fc.copy(target));
+      }
+      return fieldCollations;
+    }
+
+    protected abstract void buildSort(RelBuilder relBuilder, T sortNode, Mappings.TargetMapping mapping);
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index e03e96f..ee21a1e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rel.RelVisitor;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Aggregate.Group;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Exchange;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -62,6 +63,7 @@ import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.jdbc.HiveJdbcConverter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableFunctionScan;
@@ -88,7 +90,7 @@ public class ASTConverter {
   private Aggregate        groupBy;
   private Filter           having;
   private RelNode          select;
-  private Sort             orderLimit;
+  private RelNode          orderLimit;
 
   private Schema           schema;
 
@@ -255,7 +257,7 @@ public class ASTConverter {
      * its parent.
      * 8. Limit
      */
-    convertOrderLimitToASTNode((HiveSortLimit) orderLimit);
+    convertOrderToASTNode(orderLimit);
 
     return hiveAST.getAST();
   }
@@ -275,75 +277,98 @@ public class ASTConverter {
     return node;
   }
 
-  private void convertOrderLimitToASTNode(HiveSortLimit order) {
-    if (order != null) {
-      HiveSortLimit hiveSortLimit = order;
-      if (!hiveSortLimit.getCollation().getFieldCollations().isEmpty()) {
-        // 1 Add order by token
-        ASTNode orderAst = ASTBuilder.createAST(HiveParser.TOK_ORDERBY, "TOK_ORDERBY");
-
-        schema = new Schema(hiveSortLimit);
-        Map<Integer, RexNode> obRefToCallMap = hiveSortLimit.getInputRefToCallMap();
-        RexNode obExpr;
-        ASTNode astCol;
-        for (RelFieldCollation c : hiveSortLimit.getCollation().getFieldCollations()) {
-
-          // 2 Add Direction token
-          ASTNode directionAST = c.getDirection() == RelFieldCollation.Direction.ASCENDING ? ASTBuilder
-              .createAST(HiveParser.TOK_TABSORTCOLNAMEASC, "TOK_TABSORTCOLNAMEASC") : ASTBuilder
-              .createAST(HiveParser.TOK_TABSORTCOLNAMEDESC, "TOK_TABSORTCOLNAMEDESC");
-          ASTNode nullDirectionAST;
-          // Null direction
-          if (c.nullDirection == RelFieldCollation.NullDirection.FIRST) {
-            nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_FIRST, "TOK_NULLS_FIRST");
-            directionAST.addChild(nullDirectionAST);
-          } else if (c.nullDirection == RelFieldCollation.NullDirection.LAST) {
-            nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_LAST, "TOK_NULLS_LAST");
-            directionAST.addChild(nullDirectionAST);
-          } else {
-            // Default
-            if (c.getDirection() == RelFieldCollation.Direction.ASCENDING) {
-              nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_FIRST, "TOK_NULLS_FIRST");
-              directionAST.addChild(nullDirectionAST);
-            } else {
-              nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_LAST, "TOK_NULLS_LAST");
-              directionAST.addChild(nullDirectionAST);
-            }
-          }
+  private void convertOrderToASTNode(RelNode node) {
+    if (node == null) {
+      return;
+    }
 
-          // 3 Convert OB expr (OB Expr is usually an input ref except for top
-          // level OB; top level OB will have RexCall kept in a map.)
-          obExpr = null;
-          if (obRefToCallMap != null) {
-            obExpr = obRefToCallMap.get(c.getFieldIndex());
-          }
+    if (node instanceof HiveSortLimit) {
+      convertOrderLimitToASTNode((HiveSortLimit) node);
+    } else if (node instanceof HiveSortExchange) {
+      convertSortToASTNode((HiveSortExchange) node);
+    }
+  }
 
-          if (obExpr != null) {
-            astCol = obExpr.accept(new RexVisitor(schema, false, order.getCluster().getRexBuilder()));
-          } else {
-            ColumnInfo cI = schema.get(c.getFieldIndex());
-            /*
-             * The RowResolver setup for Select drops Table associations. So
-             * setup ASTNode on unqualified name.
-             */
-            astCol = ASTBuilder.unqualifiedName(cI.column);
-          }
+  private void convertOrderLimitToASTNode(HiveSortLimit hiveSortLimit) {
+    List<RelFieldCollation> fieldCollations = hiveSortLimit.getCollation().getFieldCollations();
+    convertFieldCollationsToASTNode(hiveSortLimit, new Schema(hiveSortLimit), fieldCollations,
+            hiveSortLimit.getInputRefToCallMap(), HiveParser.TOK_ORDERBY, "TOK_ORDERBY");
 
-          // 4 buildup the ob expr AST
-          nullDirectionAST.addChild(astCol);
-          orderAst.addChild(directionAST);
+    RexNode offsetExpr = hiveSortLimit.getOffsetExpr();
+    RexNode fetchExpr = hiveSortLimit.getFetchExpr();
+    if (fetchExpr != null) {
+      Object offset = (offsetExpr == null) ? Integer.valueOf(0) : ((RexLiteral) offsetExpr).getValue2();
+      Object fetch = ((RexLiteral) fetchExpr).getValue2();
+      hiveAST.limit = ASTBuilder.limit(offset, fetch);
+    }
+  }
+
+  private void convertSortToASTNode(HiveSortExchange hiveSortExchange) {
+    List<RelFieldCollation> fieldCollations = hiveSortExchange.getCollation().getFieldCollations();
+    convertFieldCollationsToASTNode(hiveSortExchange, new Schema(hiveSortExchange), fieldCollations,
+            null, HiveParser.TOK_SORTBY, "TOK_SORTBY");
+  }
+
+  private void convertFieldCollationsToASTNode(
+          RelNode node, Schema schema, List<RelFieldCollation> fieldCollations, Map<Integer, RexNode> obRefToCallMap,
+          int astToken, String astText) {
+    if (fieldCollations.isEmpty()) {
+      return;
+    }
+
+    // 1 Add order/sort by token
+    ASTNode orderAst = ASTBuilder.createAST(astToken, astText);
+
+    RexNode obExpr;
+    ASTNode astCol;
+    for (RelFieldCollation c : fieldCollations) {
+
+      // 2 Add Direction token
+      ASTNode directionAST = c.getDirection() == RelFieldCollation.Direction.ASCENDING ? ASTBuilder
+              .createAST(HiveParser.TOK_TABSORTCOLNAMEASC, "TOK_TABSORTCOLNAMEASC") : ASTBuilder
+              .createAST(HiveParser.TOK_TABSORTCOLNAMEDESC, "TOK_TABSORTCOLNAMEDESC");
+      ASTNode nullDirectionAST;
+      // Null direction
+      if (c.nullDirection == RelFieldCollation.NullDirection.FIRST) {
+        nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_FIRST, "TOK_NULLS_FIRST");
+        directionAST.addChild(nullDirectionAST);
+      } else if (c.nullDirection == RelFieldCollation.NullDirection.LAST) {
+        nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_LAST, "TOK_NULLS_LAST");
+        directionAST.addChild(nullDirectionAST);
+      } else {
+        // Default
+        if (c.getDirection() == RelFieldCollation.Direction.ASCENDING) {
+          nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_FIRST, "TOK_NULLS_FIRST");
+          directionAST.addChild(nullDirectionAST);
+        } else {
+          nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_LAST, "TOK_NULLS_LAST");
+          directionAST.addChild(nullDirectionAST);
         }
-        hiveAST.order = orderAst;
       }
 
-      RexNode offsetExpr = hiveSortLimit.getOffsetExpr();
-      RexNode fetchExpr = hiveSortLimit.getFetchExpr();
-      if (fetchExpr != null) {
-        Object offset = (offsetExpr == null) ? Integer.valueOf(0) : ((RexLiteral) offsetExpr).getValue2();
-        Object fetch = ((RexLiteral) fetchExpr).getValue2();
-        hiveAST.limit = ASTBuilder.limit(offset, fetch);
+      // 3 Convert OB expr (OB Expr is usually an input ref except for top
+      // level OB; top level OB will have RexCall kept in a map.)
+      obExpr = null;
+      if (obRefToCallMap != null) {
+        obExpr = obRefToCallMap.get(c.getFieldIndex());
       }
+
+      if (obExpr != null) {
+        astCol = obExpr.accept(new RexVisitor(schema, false, node.getCluster().getRexBuilder()));
+      } else {
+        ColumnInfo cI = schema.get(c.getFieldIndex());
+        /*
+         * The RowResolver setup for Select drops Table associations. So
+         * setup ASTNode on unqualified name.
+         */
+        astCol = ASTBuilder.unqualifiedName(cI.column);
+      }
+
+      // 4 buildup the ob expr AST
+      nullDirectionAST.addChild(astCol);
+      orderAst.addChild(directionAST);
     }
+    hiveAST.order = orderAst;
   }
 
   private Schema getRowSchema(String tblAlias) {
@@ -463,11 +488,11 @@ public class ASTConverter {
         ASTConverter.this.from = node;
       } else if (node instanceof Aggregate) {
         ASTConverter.this.groupBy = (Aggregate) node;
-      } else if (node instanceof Sort) {
+      } else if (node instanceof Sort || node instanceof Exchange) {
         if (ASTConverter.this.select != null) {
           ASTConverter.this.from = node;
         } else {
-          ASTConverter.this.orderLimit = (Sort) node;
+          ASTConverter.this.orderLimit = node;
         }
       }
       /*
@@ -872,11 +897,12 @@ public class ASTConverter {
      *          Hive Sort Node
      * @return Schema
      */
-    public Schema(HiveSortLimit order) {
-      Project select = (Project) order.getInput();
-      for (String projName : select.getRowType().getFieldNames()) {
-        add(new ColumnInfo(null, projName));
-      }
+    Schema(HiveSortLimit order) {
+      this((Project) order.getInput(), null);
+    }
+
+    Schema(HiveSortExchange sort) {
+      this((Project) sort.getInput(), null);
     }
 
     public Schema(String tabAlias, List<RelDataTypeField> fieldList) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index 31619c0..fe4ecf0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -28,6 +28,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Exchange;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.Project;
@@ -47,6 +48,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
@@ -70,7 +72,7 @@ public class PlanModifierForASTConv {
       LOG.debug("Original plan for PlanModifier\n " + RelOptUtil.toString(newTopNode));
     }
 
-    if (!(newTopNode instanceof Project) && !(newTopNode instanceof Sort)) {
+    if (!(newTopNode instanceof Project) && !(newTopNode instanceof Sort) && !(newTopNode instanceof Exchange)) {
       newTopNode = introduceDerivedTable(newTopNode);
       if (LOG.isDebugEnabled()) {
         LOG.debug("Plan after top-level introduceDerivedTable\n "
@@ -174,6 +176,10 @@ public class PlanModifierForASTConv {
         if (!validSortChild((HiveSortLimit) rel)) {
           introduceDerivedTable(((HiveSortLimit) rel).getInput(), rel);
         }
+      } else if (rel instanceof HiveSortExchange) {
+        if (!validExchangeChild((HiveSortExchange) rel)) {
+          introduceDerivedTable(((HiveSortExchange) rel).getInput(), rel);
+        }
       } else if (rel instanceof HiveAggregate) {
         RelNode newParent = parent;
         if (!validGBParent(rel, parent)) {
@@ -358,6 +364,10 @@ public class PlanModifierForASTConv {
     return validChild;
   }
 
+  private static boolean validExchangeChild(HiveSortExchange sortNode) {
+    return sortNode.getInput() instanceof Project;
+  }
+
   private static boolean validSetopParent(RelNode setop, RelNode parent) {
     boolean validChild = true;
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveSortExchangeVisitor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveSortExchangeVisitor.java
index 68227db..8a784c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveSortExchangeVisitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveSortExchangeVisitor.java
@@ -53,12 +53,12 @@ class HiveSortExchangeVisitor extends HiveRelNodeVisitor<HiveSortExchange> {
     if (distribution.getType() != Type.HASH_DISTRIBUTED) {
       throw new SemanticException("Only hash distribution supported for LogicalExchange");
     }
-    ExprNodeDesc[] expressions = new ExprNodeDesc[exchangeRel.getJoinKeys().size()];
-    for (int index = 0; index < exchangeRel.getJoinKeys().size(); index++) {
-      expressions[index] = HiveOpConverterUtils.convertToExprNode(exchangeRel.getJoinKeys().get(index),
+    ExprNodeDesc[] expressions = new ExprNodeDesc[exchangeRel.getKeys().size()];
+    for (int index = 0; index < exchangeRel.getKeys().size(); index++) {
+      expressions[index] = HiveOpConverterUtils.convertToExprNode(exchangeRel.getKeys().get(index),
           exchangeRel.getInput(), inputOpAf.tabAlias, inputOpAf.vcolsInCalcite);
     }
-    exchangeRel.setJoinExpressions(expressions);
+    exchangeRel.setKeyExpressions(expressions);
 
     ReduceSinkOperator rsOp = genReduceSink(inputOpAf.inputs.get(0), tabAlias, expressions,
         -1, -1, Operation.NOT_ACID, hiveOpConverter.getHiveConf());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
index 0286d54..8d9d5ae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
@@ -103,7 +103,7 @@ class JoinVisitor extends HiveRelNodeVisitor<RelNode> {
     // 4. Extract join key expressions from HiveSortExchange
     ExprNodeDesc[][] joinExpressions = new ExprNodeDesc[inputs.length][];
     for (int i = 0; i < inputs.length; i++) {
-      joinExpressions[i] = ((HiveSortExchange) joinRel.getInput(i)).getJoinExpressions();
+      joinExpressions[i] = ((HiveSortExchange) joinRel.getInput(i)).getKeyExpressions();
     }
 
     // 5. Extract rest of join predicate info. We infer the rest of join condition
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 6589eeb..9c39cb2 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
@@ -70,6 +70,8 @@ import org.apache.calcite.rel.AbstractRelNode;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelDistribution;
+import org.apache.calcite.rel.RelDistributions;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelVisitor;
@@ -203,6 +205,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectFilterPullUp
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectJoinTransposeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectMergeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectOverIntersectRemoveRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectSortExchangeTransposeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectSortTransposeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveReduceExpressionsRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveReduceExpressionsWithStatsRule;
@@ -213,10 +216,10 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRemoveSqCountCheck;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRulesRegistry;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSemiJoinRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortJoinReduceRule;
-import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortLimitPullUpConstantsRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortLimitRemoveRule;
 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.HiveSortPullUpConstantsRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortRemoveRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortUnionReduceRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSubQueryRemoveRule;
@@ -240,7 +243,6 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.views.MaterializedViewR
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTBuilder;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter;
 import org.apache.hadoop.hive.ql.parse.type.ExprNodeTypeCheck;
-import org.apache.hadoop.hive.ql.parse.type.JoinCondTypeCheckProcFactory;
 import org.apache.hadoop.hive.ql.parse.type.JoinTypeCheckCtx;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.PlanModifierForReturnPath;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.RexNodeConverter;
@@ -951,7 +953,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
       boolean topLevelQB, boolean verbose) {
 
     if (!queryProperties.hasClusterBy() && !queryProperties.hasDistributeBy()
-        && !queryProperties.hasSortBy() && !queryProperties.hasPTF() && !queryProperties.usesScript()
+        && !(queryProperties.hasSortBy() && queryProperties.hasLimit())
+        && !queryProperties.hasPTF() && !queryProperties.usesScript()
         && queryProperties.isCBOSupportedLateralViews()) {
       // Ok to run CBO.
       return null;
@@ -966,8 +969,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
       if (queryProperties.hasDistributeBy()) {
         msg += "has distribute by; ";
       }
-      if (queryProperties.hasSortBy()) {
-        msg += "has sort by; ";
+      if (queryProperties.hasSortBy() && queryProperties.hasLimit()) {
+        msg += "has sort by with limit; ";
       }
       if (queryProperties.hasPTF()) {
         msg += "has PTF; ";
@@ -2012,7 +2015,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
       rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_JOIN);
       rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_SEMIJOIN);
       rules.add(HiveSortMergeRule.INSTANCE);
-      rules.add(HiveSortLimitPullUpConstantsRule.INSTANCE);
+      rules.add(HiveSortPullUpConstantsRule.SORT_LIMIT_INSTANCE);
+      rules.add(HiveSortPullUpConstantsRule.SORT_EXCHANGE_INSTANCE);
       rules.add(HiveUnionPullUpConstantsRule.INSTANCE);
       rules.add(HiveAggregatePullUpConstantsRule.INSTANCE);
       generatePartialProgram(program, true, HepMatchOrder.BOTTOM_UP,
@@ -2435,6 +2439,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
         // 9.2.  Introduce exchange operators below join/multijoin operators
         generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
             HiveInsertExchange4JoinRule.EXCHANGE_BELOW_JOIN, HiveInsertExchange4JoinRule.EXCHANGE_BELOW_MULTIJOIN);
+      } else {
+        generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
+                HiveProjectSortExchangeTransposeRule.INSTANCE, HiveProjectMergeRule.INSTANCE);
       }
 
       // Trigger program
@@ -3949,217 +3956,279 @@ public class CalcitePlanner extends SemanticAnalyzer {
      * @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 returnRel = null;
+                                                boolean outermostOB) throws SemanticException {
 
       QBParseInfo qbp = getQBParseInfo(qb);
       String dest = qbp.getClauseNames().iterator().next();
       ASTNode obAST = qbp.getOrderByForClause(dest);
 
-      if (obAST != null) {
-        // 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));
-          }
+      if (obAST == null) {
+        return null;
+      }
+
+      // 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));
         }
+      }
 
-        // 2. Walk through OB exprs and extract field collations and additional
-        // virtual columns needed
-        final List<RexNode> newVCLst = new ArrayList<RexNode>();
-        final List<RelFieldCollation> fieldCollations = Lists.newArrayList();
-        int fieldIndex = 0;
+      OBLogicalPlanGenState obLogicalPlanGenState = beginGenOBLogicalPlan(obAST, selPair, outermostOB);
 
-        List<Node> obASTExprLst = obAST.getChildren();
-        ASTNode obASTExpr;
-        ASTNode nullObASTExpr;
-        List<Pair<ASTNode, TypeInfo>> vcASTTypePairs = new ArrayList<Pair<ASTNode, TypeInfo>>();
-        RowResolver inputRR = relToHiveRR.get(srcRel);
-        RowResolver outputRR = new RowResolver();
-
-        RexNode rnd;
-        RexNodeConverter converter = new RexNodeConverter(cluster, srcRel.getRowType(),
-            relToHiveColNameCalcitePosMap.get(srcRel), 0, false);
-        int srcRelRecordSz = srcRel.getRowType().getFieldCount();
-
-        for (int i = 0; i < obASTExprLst.size(); i++) {
-          // 2.1 Convert AST Expr to ExprNode
-          obASTExpr = (ASTNode) obASTExprLst.get(i);
-          nullObASTExpr = (ASTNode) obASTExpr.getChild(0);
-          ASTNode ref = (ASTNode) nullObASTExpr.getChild(0);
-          Map<ASTNode, ExprNodeDesc> astToExprNDescMap = null;
-          ExprNodeDesc obExprNDesc = null;
-
-          boolean isBothByPos = HiveConf.getBoolVar(conf, ConfVars.HIVE_GROUPBY_ORDERBY_POSITION_ALIAS);
-          boolean isObyByPos = isBothByPos
-              || HiveConf.getBoolVar(conf, ConfVars.HIVE_ORDERBY_POSITION_ALIAS);
-          // replace each of the position alias in ORDERBY with the actual column
-          if (ref != null && ref.getToken().getType() == HiveParser.Number) {
-            if (isObyByPos) {
-              int pos = Integer.parseInt(ref.getText());
-              if (pos > 0 && pos <= selectOutputRR.getColumnInfos().size()) {
-                // fieldIndex becomes so simple
-                // Note that pos starts from 1 while fieldIndex starts from 0;
-                fieldIndex = pos - 1;
-              } else {
-                throw new SemanticException(
-                    ErrorMsg.INVALID_POSITION_ALIAS_IN_ORDERBY.getMsg("Position alias: " + pos
-                        + " does not exist\n" + "The Select List is indexed from 1 to "
-                        + selectOutputRR.getColumnInfos().size()));
-              }
-            } else { // if not using position alias and it is a number.
-              LOG.warn("Using constant number "
-                  + ref.getText()
-                  + " in order by. If you try to use position alias when hive.orderby.position.alias is false, the position alias will be ignored.");
-            }
-          } else {
-            // 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);
-
-            // 2.3 Determine the index of ob expr in child schema
-            // NOTE: Calcite can not take compound exprs in OB without it being
-            // present in the child (& hence we add a child Project Rel)
-            if (rnd instanceof RexInputRef) {
-              fieldIndex = ((RexInputRef) rnd).getIndex();
-            } else {
-              fieldIndex = srcRelRecordSz + newVCLst.size();
-              newVCLst.add(rnd);
-              vcASTTypePairs.add(new Pair<ASTNode, TypeInfo>(ref, obExprNDesc.getTypeInfo()));
-            }
-          }
+      // 4. Construct SortRel
+      RelTraitSet traitSet = cluster.traitSetOf(HiveRelNode.CONVENTION);
+      RelCollation canonizedCollation = traitSet.canonize(
+              RelCollationImpl.of(obLogicalPlanGenState.getFieldCollation()));
+      RelNode sortRel = new HiveSortLimit(
+              cluster, traitSet, obLogicalPlanGenState.getObInputRel(), canonizedCollation, null, null);
+
+      return endGenOBLogicalPlan(obLogicalPlanGenState, sortRel);
+    }
+
+    private RelNode genSBLogicalPlan(QB qb, Pair<RelNode, RowResolver> selPair,
+                                   boolean outermostOB) throws SemanticException {
+
+      QBParseInfo qbp = getQBParseInfo(qb);
+      String dest = qbp.getClauseNames().iterator().next();
+      ASTNode sbAST = qbp.getSortByForClause(dest);
 
-          // 2.4 Determine the Direction of order by
-          RelFieldCollation.Direction order = RelFieldCollation.Direction.DESCENDING;
-          if (obASTExpr.getType() == HiveParser.TOK_TABSORTCOLNAMEASC) {
-            order = RelFieldCollation.Direction.ASCENDING;
+      if (sbAST == null) {
+        return null;
+      }
+
+      OBLogicalPlanGenState obLogicalPlanGenState = beginGenOBLogicalPlan(sbAST, selPair, outermostOB);
+
+      // 4. Construct SortRel
+      RelTraitSet traitSet = cluster.traitSetOf(HiveRelNode.CONVENTION);
+      RelCollation canonizedCollation =
+              traitSet.canonize(RelCollationImpl.of(obLogicalPlanGenState.getFieldCollation()));
+      List<Integer> joinKeyPositions = new ArrayList<>(canonizedCollation.getFieldCollations().size());
+      ImmutableList.Builder<RexNode> builder = ImmutableList.builder();
+      for (RelFieldCollation relFieldCollation : canonizedCollation.getFieldCollations()) {
+        int index = relFieldCollation.getFieldIndex();
+        joinKeyPositions.add(index);
+        builder.add(cluster.getRexBuilder().makeInputRef(obLogicalPlanGenState.getObInputRel(), index));
+      }
+
+      RelNode sortRel = HiveSortExchange.create(
+                  obLogicalPlanGenState.getObInputRel(),
+                  // In case of SORT BY we do not need Distribution
+                  // but the instance RelDistributions.ANY can not be used here because
+                  // org.apache.calcite.rel.core.Exchange has
+                  // assert distribution != RelDistributions.ANY;
+                  new HiveRelDistribution(RelDistribution.Type.ANY, RelDistributions.ANY.getKeys()),
+              canonizedCollation,
+              builder.build());
+
+      return endGenOBLogicalPlan(obLogicalPlanGenState, sortRel);
+    }
+
+    // - Walk through OB exprs and extract field collations and additional virtual columns needed
+    // - Add Child Project Rel if needed,
+    // - Generate Output RR, input Sel Rel for top constraining Sel
+    private OBLogicalPlanGenState beginGenOBLogicalPlan(
+            ASTNode obAST, 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();
+
+      // 2. Walk through OB exprs and extract field collations and additional
+      // virtual columns needed
+      final List<RexNode> newVCLst = new ArrayList<>();
+      final List<RelFieldCollation> fieldCollations = Lists.newArrayList();
+      int fieldIndex = 0;
+
+      List<Node> obASTExprLst = obAST.getChildren();
+      List<Pair<ASTNode, TypeInfo>> vcASTTypePairs = new ArrayList<>();
+      RowResolver inputRR = relToHiveRR.get(srcRel);
+      RowResolver outputRR = new RowResolver();
+
+      RexNode rnd;
+      RexNodeConverter converter = new RexNodeConverter(cluster, srcRel.getRowType(),
+              relToHiveColNameCalcitePosMap.get(srcRel), 0, false);
+      int srcRelRecordSz = srcRel.getRowType().getFieldCount();
+
+      for (int i = 0; i < obASTExprLst.size(); i++) {
+        // 2.1 Convert AST Expr to ExprNode
+        ASTNode obASTExpr = (ASTNode) obASTExprLst.get(i);
+        ASTNode nullObASTExpr = (ASTNode) obASTExpr.getChild(0);
+        ASTNode ref = (ASTNode) nullObASTExpr.getChild(0);
+
+        boolean isBothByPos = HiveConf.getBoolVar(conf, ConfVars.HIVE_GROUPBY_ORDERBY_POSITION_ALIAS);
+        boolean isObyByPos = isBothByPos
+                || HiveConf.getBoolVar(conf, ConfVars.HIVE_ORDERBY_POSITION_ALIAS);
+        // replace each of the position alias in ORDERBY with the actual column
+        if (ref != null && ref.getToken().getType() == HiveParser.Number) {
+          if (isObyByPos) {
+            fieldIndex = getFieldIndexFromColumnNumber(selectOutputRR, ref);
+          } else { // if not using position alias and it is a number.
+            LOG.warn("Using constant number "
+                    + ref.getText()
+                    + " in order by. If you try to use position alias when hive.orderby.position.alias is false, " +
+                    "the position alias will be ignored.");
           }
-          RelFieldCollation.NullDirection nullOrder;
-          if (nullObASTExpr.getType() == HiveParser.TOK_NULLS_FIRST) {
-            nullOrder = RelFieldCollation.NullDirection.FIRST;
-          } else if (nullObASTExpr.getType() == HiveParser.TOK_NULLS_LAST) {
-            nullOrder = RelFieldCollation.NullDirection.LAST;
+        } else {
+          ExprNodeDesc obExprNDesc = getOrderByExprNodeDesc(selectOutputRR, inputRR, obASTExpr, ref);
+          // 2.2 Convert ExprNode to RexNode
+          rnd = converter.convert(obExprNDesc);
+
+          // 2.3 Determine the index of ob expr in child schema
+          // NOTE: Calcite can not take compound exprs in OB without it being
+          // present in the child (& hence we add a child Project Rel)
+          if (rnd instanceof RexInputRef) {
+            fieldIndex = ((RexInputRef) rnd).getIndex();
           } else {
-            throw new SemanticException("Unexpected null ordering option: "
-                + nullObASTExpr.getType());
+            fieldIndex = srcRelRecordSz + newVCLst.size();
+            newVCLst.add(rnd);
+            vcASTTypePairs.add(new Pair<>(ref, obExprNDesc.getTypeInfo()));
           }
+        }
 
-          // 2.5 Add to field collations
-          fieldCollations.add(new RelFieldCollation(fieldIndex, order, nullOrder));
+        // 2.4 Determine the Direction of order by
+        RelFieldCollation.Direction order = RelFieldCollation.Direction.DESCENDING;
+        if (obASTExpr.getType() == HiveParser.TOK_TABSORTCOLNAMEASC) {
+          order = RelFieldCollation.Direction.ASCENDING;
+        }
+        RelFieldCollation.NullDirection nullOrder;
+        if (nullObASTExpr.getType() == HiveParser.TOK_NULLS_FIRST) {
+          nullOrder = RelFieldCollation.NullDirection.FIRST;
+        } else if (nullObASTExpr.getType() == HiveParser.TOK_NULLS_LAST) {
+          nullOrder = RelFieldCollation.NullDirection.LAST;
+        } else {
+          throw new SemanticException("Unexpected null ordering option: "
+                  + nullObASTExpr.getType());
         }
 
-        // 3. Add Child Project Rel if needed, Generate Output RR, input Sel Rel
-        // for top constraining Sel
-        RelNode obInputRel = srcRel;
-        if (!newVCLst.isEmpty()) {
-          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());
-                }
-              });
-          RowResolver obSyntheticProjectRR = new RowResolver();
-          if (!RowResolver.add(obSyntheticProjectRR, inputRR)) {
-            throw new CalciteSemanticException(
-                "Duplicates detected when adding columns to RR: see previous message",
-                UnsupportedFeature.Duplicates_in_RR);
-          }
-          int vcolPos = inputRR.getRowSchema().getSignature().size();
-          for (Pair<ASTNode, TypeInfo> astTypePair : vcASTTypePairs) {
-            obSyntheticProjectRR.putExpression(astTypePair.getKey(), new ColumnInfo(
-                SemanticAnalyzer.getColumnInternalName(vcolPos), astTypePair.getValue(), null,
-                false));
-            vcolPos++;
-          }
-          obInputRel = genSelectRelNode(CompositeList.of(originalInputRefs, newVCLst),
-              obSyntheticProjectRR, srcRel);
+        // 2.5 Add to field collations
+        fieldCollations.add(new RelFieldCollation(fieldIndex, order, nullOrder));
+      }
 
-          if (outermostOB) {
-            if (!RowResolver.add(outputRR, inputRR)) {
-              throw new CalciteSemanticException(
+      // 3. Add Child Project Rel if needed, Generate Output RR, input Sel Rel
+      // for top constraining Sel
+      RelNode obInputRel = srcRel;
+      if (!newVCLst.isEmpty()) {
+        List<RexNode> originalInputRefs = toRexNodeList(srcRel);
+        RowResolver obSyntheticProjectRR = new RowResolver();
+        if (!RowResolver.add(obSyntheticProjectRR, inputRR)) {
+          throw new CalciteSemanticException(
                   "Duplicates detected when adding columns to RR: see previous message",
                   UnsupportedFeature.Duplicates_in_RR);
-            }
+        }
+        int vcolPos = inputRR.getRowSchema().getSignature().size();
+        for (Pair<ASTNode, TypeInfo> astTypePair : vcASTTypePairs) {
+          obSyntheticProjectRR.putExpression(astTypePair.getKey(), new ColumnInfo(
+                  SemanticAnalyzer.getColumnInternalName(vcolPos), astTypePair.getValue(), null,
+                  false));
+          vcolPos++;
+        }
+        obInputRel = genSelectRelNode(CompositeList.of(originalInputRefs, newVCLst),
+                obSyntheticProjectRR, srcRel);
 
-          } else {
-            if (!RowResolver.add(outputRR, obSyntheticProjectRR)) {
-              throw new CalciteSemanticException(
-                  "Duplicates detected when adding columns to RR: see previous message",
-                  UnsupportedFeature.Duplicates_in_RR);
-            }
+        if (outermostOB) {
+          if (!RowResolver.add(outputRR, inputRR)) {
+            throw new CalciteSemanticException(
+                    "Duplicates detected when adding columns to RR: see previous message",
+                    UnsupportedFeature.Duplicates_in_RR);
           }
+
         } else {
-          if (!RowResolver.add(outputRR, inputRR)) {
+          if (!RowResolver.add(outputRR, obSyntheticProjectRR)) {
             throw new CalciteSemanticException(
-                "Duplicates detected when adding columns to RR: see previous message",
-                UnsupportedFeature.Duplicates_in_RR);
+                    "Duplicates detected when adding columns to RR: see previous message",
+                    UnsupportedFeature.Duplicates_in_RR);
           }
         }
+      } else {
+        if (!RowResolver.add(outputRR, inputRR)) {
+          throw new CalciteSemanticException(
+                  "Duplicates detected when adding columns to RR: see previous message",
+                  UnsupportedFeature.Duplicates_in_RR);
+        }
+      }
+      return new OBLogicalPlanGenState(obInputRel, fieldCollations, selectOutputRR, outputRR, srcRel);
+    }
 
-        // 4. Construct SortRel
-        RelTraitSet traitSet = cluster.traitSetOf(HiveRelNode.CONVENTION);
-        RelCollation canonizedCollation = traitSet.canonize(RelCollationImpl.of(fieldCollations));
-        sortRel = new HiveSortLimit(cluster, traitSet, obInputRel, canonizedCollation, null, null);
-
-        // 5. Update the maps
-        // NOTE: Output RR for SortRel is considered same as its input; we may
-        // end up not using VC that is present in sort rel. Also note that
-        // rowtype of sortrel is the type of it child; if child happens to be
-        // synthetic project that we introduced then that projectrel would
-        // contain the vc.
-        ImmutableMap<String, Integer> hiveColNameCalcitePosMap = buildHiveToCalciteColumnMap(outputRR);
-        relToHiveRR.put(sortRel, outputRR);
-        relToHiveColNameCalcitePosMap.put(sortRel, hiveColNameCalcitePosMap);
+    private ExprNodeDesc getOrderByExprNodeDesc(
+            RowResolver selectOutputRR, RowResolver inputRR, ASTNode obASTExpr, ASTNode ref)
+            throws SemanticException {
+      // first try to get it from select
+      // in case of udtf, selectOutputRR may be null.
+      ExprNodeDesc obExprNDesc = null;
+      if (selectOutputRR != null) {
+        try {
+          Map<ASTNode, ExprNodeDesc> 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) {
+        Map<ASTNode, ExprNodeDesc> astToExprNDescMap = genAllExprNodeDesc(ref, inputRR);
+        obExprNDesc = astToExprNDescMap.get(ref);
+      }
+      if (obExprNDesc == null) {
+        throw new SemanticException("Invalid order by expression: " + obASTExpr.toString());
+      }
+      return obExprNDesc;
+    }
 
-        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;
+    // SELECT a, b FROM t ORDER BY 1
+    private int getFieldIndexFromColumnNumber(RowResolver selectOutputRR, ASTNode ref) throws SemanticException {
+      int fieldIndex;
+      int pos = Integer.parseInt(ref.getText());
+      if (pos > 0 && pos <= selectOutputRR.getColumnInfos().size()) {
+        // fieldIndex becomes so simple
+        // Note that pos starts from 1 while fieldIndex starts from 0;
+        fieldIndex = pos - 1;
+      } else {
+        throw new SemanticException(
+                ErrorMsg.INVALID_POSITION_ALIAS_IN_ORDERBY.getMsg("Position alias: " + pos
+                        + " does not exist\n" + "The Select List is indexed from 1 to "
+                        + selectOutputRR.getColumnInfos().size()));
+      }
+      return fieldIndex;
+    }
+
+    private List<RexNode> toRexNodeList(RelNode srcRel) {
+      return srcRel.getRowType().getFieldList().stream()
+              .map(input -> new RexInputRef(input.getIndex(), input.getType()))
+              .collect(Collectors.toList());
+    }
+
+    // 5. Update RR maps
+    // NOTE: Output RR for SortRel is considered same as its input; we may
+    // end up not using VC that is present in sort rel. Also note that
+    // rowtype of sortrel is the type of it child; if child happens to be
+    // synthetic project that we introduced then that projectrel would
+    // contain the vc.
+    public RelNode endGenOBLogicalPlan(OBLogicalPlanGenState obLogicalPlanGenState, RelNode sortRel)
+            throws CalciteSemanticException {
+
+      ImmutableMap<String, Integer> hiveColNameCalcitePosMap =
+              buildHiveToCalciteColumnMap(obLogicalPlanGenState.getOutputRR());
+      relToHiveRR.put(sortRel, obLogicalPlanGenState.getOutputRR());
+      relToHiveColNameCalcitePosMap.put(sortRel, hiveColNameCalcitePosMap);
+
+      if (obLogicalPlanGenState.getSelectOutputRR() != null) {
+        List<RexNode> originalInputRefs = toRexNodeList(obLogicalPlanGenState.getSrcRel());
+        List<RexNode> selectedRefs = Lists.newArrayList();
+        for (int index = 0; index < obLogicalPlanGenState.getSelectOutputRR().getColumnInfos().size(); index++) {
+          selectedRefs.add(originalInputRefs.get(index));
         }
+        // We need to add select since order by schema may have more columns than result schema.
+        return genSelectRelNode(selectedRefs, obLogicalPlanGenState.getSelectOutputRR(), sortRel);
+      } else {
+        return sortRel;
       }
-      return returnRel;
     }
 
     private RelNode genLimitLogicalPlan(QB qb, RelNode srcRel) throws SemanticException {
@@ -4771,6 +4840,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       } else {
         String dest = qbp.getClauseNames().iterator().next();
         ASTNode obAST = qbp.getOrderByForClause(dest);
+        ASTNode sbAST = qbp.getSortByForClause(dest);
 
         RowResolver originalRR = null;
         // We only support limited unselected column following by order by.
@@ -4781,7 +4851,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         // If DISTINCT is present, it is not possible to ORDER BY unselected
         // columns, and in fact adding all columns would change the behavior of
         // DISTINCT, so we bypass this logic.
-        if (obAST != null
+        if ((obAST != null || sbAST != null)
             && selExprList.getToken().getType() != HiveParser.TOK_SELECTDI
             && !isAllColRefRewrite) {
           // 1. OB Expr sanity test
@@ -5006,6 +5076,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
       RelNode gbHavingRel = null;
       RelNode selectRel = null;
       RelNode obRel = null;
+      RelNode sbRel = null;
       RelNode limitRel = null;
 
       // First generate all the opInfos for the elements in the from clause
@@ -5097,11 +5168,15 @@ public class CalcitePlanner extends SemanticAnalyzer {
       obRel = genOBLogicalPlan(qb, selPair, outerMostQB);
       srcRel = (obRel == null) ? srcRel : obRel;
 
-      // 7. Build Rel for Limit Clause
+      // 7. Build Rel for Sort By Clause
+      sbRel = genSBLogicalPlan(qb, selPair, outerMostQB);
+      srcRel = (sbRel == null) ? srcRel : sbRel;
+
+      // 8. Build Rel for Limit Clause
       limitRel = genLimitLogicalPlan(qb, srcRel);
       srcRel = (limitRel == null) ? srcRel : limitRel;
 
-      // 8. Incase this QB corresponds to subquery then modify its RR to point
+      // 9. Incase this QB corresponds to subquery then modify its RR to point
       // to subquery alias.
       if (qb.getParseInfo().getAlias() != null) {
         RowResolver rr = this.relToHiveRR.get(srcRel);
@@ -5244,6 +5319,47 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
   }
 
+  /**
+   * This class stores the partial results of Order/Sort by clause logical plan generation.
+   * See {@link CalcitePlannerAction#beginGenOBLogicalPlan}, {@link CalcitePlannerAction#endGenOBLogicalPlan}
+   */
+  private static class OBLogicalPlanGenState {
+    private final RelNode obInputRel;
+    private final List<RelFieldCollation> canonizedCollation;
+    private final RowResolver selectOutputRR;
+    private final RowResolver outputRR;
+    private final RelNode srcRel;
+
+    OBLogicalPlanGenState(RelNode obInputRel, List<RelFieldCollation> canonizedCollation,
+                                 RowResolver selectOutputRR, RowResolver outputRR, RelNode srcRel) {
+      this.obInputRel = obInputRel;
+      this.canonizedCollation = canonizedCollation;
+      this.selectOutputRR = selectOutputRR;
+      this.outputRR = outputRR;
+      this.srcRel = srcRel;
+    }
+
+    public RelNode getObInputRel() {
+      return obInputRel;
+    }
+
+    public List<RelFieldCollation> getFieldCollation() {
+      return canonizedCollation;
+    }
+
+    public RowResolver getSelectOutputRR() {
+      return selectOutputRR;
+    }
+
+    public RowResolver getOutputRR() {
+      return outputRR;
+    }
+
+    public RelNode getSrcRel() {
+      return srcRel;
+    }
+  }
+
   @Override
   protected Table getTableObjectByName(String tabName, boolean throwException) throws HiveException {
     String[] names = Utilities.getDbTableName(tabName);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java
index 31068cb..7b25030 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/RewriteSemanticAnalyzer.java
@@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory;
  * statements (since they are actually inserts) and then doing some patch up to make them work as
  * updates and deletes instead.
  */
-public abstract class RewriteSemanticAnalyzer extends SemanticAnalyzer {
+public abstract class RewriteSemanticAnalyzer extends CalcitePlanner {
   protected static final Logger LOG = LoggerFactory.getLogger(RewriteSemanticAnalyzer.class);
 
   protected boolean useSuper = false;
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 98317d1..c87f2d2 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
@@ -811,20 +811,18 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       ASTNode selectExpr = (ASTNode) selectExprs.getChild(i);
       if (selectExpr.getChildCount() == 1 && selectExpr.getChild(0).getType() == HiveParser.TOK_TABLE_OR_COL) {
         //first child should be rowid
-        if (i == 0 && !selectExpr.getChild(0).getChild(0).getText().equals("ROW__ID")) {
-          throw new SemanticException("Unexpected element when replacing default keyword for UPDATE."
-                                          + " Expected ROW_ID, found: " + selectExpr.getChild(0).getChild(0).getText());
-        }
-        else if (selectExpr.getChild(0).getChild(0).getText().toLowerCase().equals("default")) {
-          if (defaultConstraints == null) {
-            defaultConstraints = getDefaultConstraints(targetTable, null);
-          }
-          ASTNode newNode = getNodeReplacementforDefault(defaultConstraints.get(i - 1));
-          // replace the node in place
-          selectExpr.replaceChildren(0, 0, newNode);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("DEFAULT keyword replacement - Inserted {} for table: {}", newNode.getText(),
-                targetTable.getTableName());
+        if (i != 0 || selectExpr.getChild(0).getChild(0).getText().equals("ROW__ID")) {
+          if (selectExpr.getChild(0).getChild(0).getText().toLowerCase().equals("default")) {
+            if (defaultConstraints == null) {
+              defaultConstraints = getDefaultConstraints(targetTable, null);
+            }
+            ASTNode newNode = getNodeReplacementforDefault(defaultConstraints.get(i - 1));
+            // replace the node in place
+            selectExpr.replaceChildren(0, 0, newNode);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("DEFAULT keyword replacement - Inserted {} for table: {}", newNode.getText(),
+                  targetTable.getTableName());
+            }
           }
         }
       }
@@ -1809,6 +1807,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         break;
 
       case HiveParser.TOK_LIMIT:
+        queryProperties.setHasLimit(true);
         if (ast.getChildCount() == 2) {
           qbp.setDestLimit(ctx_1.dest,
               Integer.valueOf(ast.getChild(0).getText()), Integer.valueOf(ast.getChild(1).getText()));
diff --git a/ql/src/test/queries/clientpositive/authorization_view_disable_cbo_1.q b/ql/src/test/queries/clientpositive/authorization_view_disable_cbo_1.q
index be50b69..f228ccd 100644
--- a/ql/src/test/queries/clientpositive/authorization_view_disable_cbo_1.q
+++ b/ql/src/test/queries/clientpositive/authorization_view_disable_cbo_1.q
@@ -64,7 +64,7 @@ set hive.cbo.enable=true;
 
 --although cbo is enabled, it will not succeed.
 
-select key from v_n10 sort by key limit 10;
+select key from v_n10 cluster by key limit 10;
 
 select key from
 (select key as key from src_autho_test_n9 union all select key from v_n10 cluster by key)subq
diff --git a/ql/src/test/queries/clientpositive/sort.q b/ql/src/test/queries/clientpositive/sort.q
index cab2712..d8c8969 100644
--- a/ql/src/test/queries/clientpositive/sort.q
+++ b/ql/src/test/queries/clientpositive/sort.q
@@ -5,3 +5,8 @@ EXPLAIN
 SELECT x.* FROM SRC x SORT BY key;
 
 SELECT x.* FROM SRC x SORT BY key;
+
+EXPLAIN
+SELECT x.* FROM SRC x SORT BY 1;
+
+SELECT x.* FROM SRC x SORT BY 1;
diff --git a/ql/src/test/queries/clientpositive/sort_acid.q b/ql/src/test/queries/clientpositive/sort_acid.q
new file mode 100644
index 0000000..9d0b9d0
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/sort_acid.q
@@ -0,0 +1,35 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create table acidtlb(a int, b int) clustered by (a) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
+create table othertlb(c int, d int) stored as orc TBLPROPERTIES ('transactional'='true');
+
+insert into acidtlb values(10,200),(30,500);
+insert into othertlb values(10, 21),(30, 22),(60, 23),(70, 24),(80, 25);
+
+
+explain cbo
+select a, 6 as c, b from acidtlb sort by a, c, b;
+select a, 6 as c, b from acidtlb sort by a, c, b;
+
+update acidtlb set b=777;
+select * from acidtlb;
+
+
+update acidtlb set b=350
+where a in (select a from acidtlb where a = 30);
+
+select * from acidtlb;
+
+update acidtlb set b=450
+where a in (select c from othertlb where c < 65);
+
+select * from acidtlb;
+
+delete from acidtlb
+where a in (
+    select a from acidtlb a
+             join othertlb o on a.a = o.c
+             where o.d = 21);
+
+select * from acidtlb;
diff --git a/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite.q.out b/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite.q.out
index 2b7ff65..8e55c2c 100644
--- a/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite.q.out
@@ -19,4 +19,4 @@ POSTHOOK: Output: default@cmv_basetable
 POSTHOOK: Lineage: cmv_basetable.a SCRIPT []
 POSTHOOK: Lineage: cmv_basetable.b SCRIPT []
 POSTHOOK: Lineage: cmv_basetable.c SCRIPT []
-FAILED: SemanticException Cannot enable automatic rewriting for materialized view. Plan not optimized by CBO because the statement has sort by
+FAILED: SemanticException Cannot enable automatic rewriting for materialized view. Unsupported RelNode type HiveSortExchange encountered in the query plan
diff --git a/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite_2.q.out b/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite_2.q.out
index 6850290..b9de681 100644
--- a/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite_2.q.out
+++ b/ql/src/test/results/clientnegative/materialized_view_no_cbo_rewrite_2.q.out
@@ -33,4 +33,4 @@ PREHOOK: query: alter materialized view cmv_mat_view enable rewrite
 PREHOOK: type: ALTER_MATERIALIZED_VIEW_REWRITE
 PREHOOK: Input: default@cmv_mat_view
 PREHOOK: Output: default@cmv_mat_view
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.ddl.DDLTask. org.apache.hadoop.hive.ql.metadata.HiveException: Cannot enable automatic rewriting for materialized view. Plan not optimized by CBO because the statement has sort by
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.ddl.DDLTask. org.apache.hadoop.hive.ql.metadata.HiveException: Cannot enable rewriting for materialized view. Unsupported RelNode type HiveSortExchange encountered in the query plan
diff --git a/ql/src/test/results/clientnegative/update_notnull_constraint.q.out b/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
index 86bfc67..3290537 100644
--- a/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
+++ b/ql/src/test/results/clientnegative/update_notnull_constraint.q.out
@@ -21,4 +21,9 @@ POSTHOOK: Output: default@acid_uami
 POSTHOOK: Lineage: acid_uami.de SCRIPT []
 POSTHOOK: Lineage: acid_uami.i SCRIPT []
 POSTHOOK: Lineage: acid_uami.vc SCRIPT []
-FAILED: DataConstraintViolationError org.apache.hadoop.hive.ql.exec.errors.DataConstraintViolationError: Either CHECK or NOT NULL constraint violated!
+PREHOOK: query: UPDATE acid_uami set de=null where i=1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@acid_uami
+PREHOOK: Output: default@acid_uami
+#### A masked pattern was here ####
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask
diff --git a/ql/src/test/results/clientpositive/acid_view_delete.q.out b/ql/src/test/results/clientpositive/acid_view_delete.q.out
index 3771c3b..a1de850 100644
--- a/ql/src/test/results/clientpositive/acid_view_delete.q.out
+++ b/ql/src/test/results/clientpositive/acid_view_delete.q.out
@@ -94,7 +94,7 @@ POSTHOOK: Input: default@mydim
 #### A masked pattern was here ####
 1	bob	95136	true
 2	joe	70068	true
-Warning: Shuffle Join JOIN[20][tables = [mydim, sq_1_notin_nullcheck]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: update mydim set is_current = false
 where mydim.key not in(select kv from updates_staging_view)
 PREHOOK: type: QUERY
diff --git a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out
index b609982..0b50d93 100644
--- a/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out
+++ b/ql/src/test/results/clientpositive/authorization_view_disable_cbo_1.q.out
@@ -259,12 +259,12 @@ val_11
 val_111
 val_113
 val_114
-PREHOOK: query: select key from v_n10 sort by key limit 10
+PREHOOK: query: select key from v_n10 cluster by key limit 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src_autho_test_n9
 PREHOOK: Input: default@v_n10
 #### A masked pattern was here ####
-POSTHOOK: query: select key from v_n10 sort by key limit 10
+POSTHOOK: query: select key from v_n10 cluster by key limit 10
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src_autho_test_n9
 POSTHOOK: Input: default@v_n10
diff --git a/ql/src/test/results/clientpositive/auto_join0.q.out b/ql/src/test/results/clientpositive/auto_join0.q.out
index 665cf28..12fe541 100644
--- a/ql/src/test/results/clientpositive/auto_join0.q.out
+++ b/ql/src/test/results/clientpositive/auto_join0.q.out
@@ -1,6 +1,6 @@
-Warning: Map Join MAPJOIN[35][bigTable=?] in task 'Stage-5:MAPRED' is a cross product
-Warning: Map Join MAPJOIN[43][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[12][tables = [src1, src2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[38][bigTable=?] in task 'Stage-5:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[47][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain 
 select sum(hash(a.k1,a.v1,a.k2, a.v2))
 from (
@@ -45,10 +45,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string)
@@ -78,11 +78,11 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        a:$INTNAME1 
+        $hdt$_0:$INTNAME1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        a:$INTNAME1 
+        $hdt$_0:$INTNAME1 
           TableScan
             HashTable Sink Operator
               keys:
@@ -101,18 +101,22 @@ STAGE PLANS:
                 1 
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
-              Group By Operator
-                aggregations: sum(hash(_col0,_col1,_col2,_col3))
-                minReductionHashAggr: 0.99
-                mode: hash
+              Select Operator
+                expressions: hash(_col0,_col1,_col2,_col3) (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 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
+                Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -144,11 +148,11 @@ STAGE PLANS:
   Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        a:$INTNAME 
+        $hdt$_0:$INTNAME 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        a:$INTNAME 
+        $hdt$_0:$INTNAME 
           TableScan
             HashTable Sink Operator
               keys:
@@ -167,18 +171,22 @@ STAGE PLANS:
                 1 
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
-              Group By Operator
-                aggregations: sum(hash(_col0,_col1,_col2,_col3))
-                minReductionHashAggr: 0.99
-                mode: hash
+              Select Operator
+                expressions: hash(_col0,_col1,_col2,_col3) (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 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
+                Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -207,28 +215,32 @@ STAGE PLANS:
             1 
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
-          Group By Operator
-            aggregations: sum(hash(_col0,_col1,_col2,_col3))
-            minReductionHashAggr: 0.99
-            mode: hash
+          Select Operator
+            expressions: hash(_col0,_col1,_col2,_col3) (type: int)
             outputColumnNames: _col0
-            Statistics: Num rows: 1 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
+            Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
+            Group By Operator
+              aggregations: sum(_col0)
+              minReductionHashAggr: 0.99
+              mode: hash
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 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-4
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string)
@@ -258,9 +270,9 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[35][bigTable=?] in task 'Stage-5:MAPRED' is a cross product
-Warning: Map Join MAPJOIN[43][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[12][tables = [src1, src2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[38][bigTable=?] in task 'Stage-5:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[47][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: select sum(hash(a.k1,a.v1,a.k2, a.v2))
 from (
 SELECT src1.key as k1, src1.value as v1, 
diff --git a/ql/src/test/results/clientpositive/auto_join15.q.out b/ql/src/test/results/clientpositive/auto_join15.q.out
index 6ea9db2..15fa1d2 100644
--- a/ql/src/test/results/clientpositive/auto_join15.q.out
+++ b/ql/src/test/results/clientpositive/auto_join15.q.out
@@ -27,11 +27,11 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        a:src1 
+        $hdt$_0:$hdt$_0:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        a:src1 
+        $hdt$_0:$hdt$_0:src1 
           TableScan
             alias: src1
             filterExpr: key is not null (type: boolean)
@@ -39,10 +39,14 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-2
     Map Reduce
@@ -54,29 +58,33 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6
-                Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    aggregations: sum(hash(_col0,_col1,_col2,_col3))
-                    minReductionHashAggr: 0.99
-                    mode: hash
+                  Select Operator
+                    expressions: hash(_col0,_col1,_col2,_col3) (type: int)
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      null sort order: 
-                      sort order: 
+                    Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: sum(_col0)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: bigint)
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: bigint)
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
diff --git a/ql/src/test/results/clientpositive/auto_join20.q.out b/ql/src/test/results/clientpositive/auto_join20.q.out
index 6bbcb47..6475ad2 100644
--- a/ql/src/test/results/clientpositive/auto_join20.q.out
+++ b/ql/src/test/results/clientpositive/auto_join20.q.out
@@ -32,47 +32,55 @@ STAGE PLANS:
   Stage: Stage-10
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        a:src1 
+        $hdt$_0:$hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        a:src1 
+        $hdt$_0:$hdt$_1:src1 
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-8
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6
-                Statistics: Num rows: 262 Data size: 93272 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
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 166 Data size: 59096 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -83,19 +91,19 @@ STAGE PLANS:
   Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        a:$INTNAME 
+        $hdt$_0:$INTNAME 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        a:$INTNAME 
+        $hdt$_0:$INTNAME 
           TableScan
             HashTable Sink Operator
               filter predicates:
                 0 
-                1 {(key < 20)}
+                1 {_col2}
               keys:
                 0 _col0 (type: string)
-                1 key (type: string)
+                1 _col0 (type: string)
 
   Stage: Stage-6
     Map Reduce
@@ -103,33 +111,37 @@ STAGE PLANS:
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key < 20)}
-              keys:
-                0 _col0 (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-              Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 20.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {_col2}
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  aggregations: sum(hash(_col0,_col1,_col2,_col3,_col4,_col5))
-                  minReductionHashAggr: 0.99
-                  mode: hash
+                Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: hash(_col0,_col1,_col2,_col3,_col4,_col5) (type: int)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 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
+                  Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: sum(_col0)
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -167,36 +179,40 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 20.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 20)}
+            1 {VALUE._col1}
           keys:
             0 _col0 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
+            expressions: hash(_col0,_col1,_col2,_col3,_col4,_col5) (type: int)
+            outputColumnNames: _col0
+            Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
-              aggregations: sum(hash(_col0,_col1,_col2,_col3,_col4,_col5))
+              aggregations: sum(_col0)
               minReductionHashAggr: 0.99
               mode: hash
               outputColumnNames: _col0
@@ -267,47 +283,55 @@ STAGE PLANS:
   Stage: Stage-10
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        a:src1 
+        $hdt$_0:$hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        a:src1 
+        $hdt$_0:$hdt$_1:src1 
           TableScan
             alias: src1
-            filterExpr: ((key < 15) and (key < 10)) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: ((key < 15) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-8
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
-            filterExpr: ((key < 10) and (key < 15)) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: ((key < 10) and (key < 15)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6
-                Statistics: Num rows: 86 Data size: 30616 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
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 166 Data size: 59096 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -318,19 +342,19 @@ STAGE PLANS:
   Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        a:$INTNAME 
+        $hdt$_0:$INTNAME 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        a:$INTNAME 
+        $hdt$_0:$INTNAME 
           TableScan
             HashTable Sink Operator
               filter predicates:
                 0 
-                1 {(key < 20)}
+                1 {_col2}
               keys:
                 0 _col0 (type: string)
-                1 key (type: string)
+                1 _col0 (type: string)
 
   Stage: Stage-6
     Map Reduce
@@ -338,33 +362,37 @@ STAGE PLANS:
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key < 20)}
-              keys:
-                0 _col0 (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-              Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 20.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {_col2}
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  aggregations: sum(hash(_col0,_col1,_col2,_col3,_col4,_col5))
-                  minReductionHashAggr: 0.99
-                  mode: hash
+                Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: hash(_col0,_col1,_col2,_col3,_col4,_col5) (type: int)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 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
+                  Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: sum(_col0)
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -402,36 +430,40 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 20.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 20)}
+            1 {VALUE._col1}
           keys:
             0 _col0 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+            expressions: hash(_col0,_col1,_col2,_col3,_col4,_col5) (type: int)
+            outputColumnNames: _col0
+            Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
-              aggregations: sum(hash(_col0,_col1,_col2,_col3,_col4,_col5))
+              aggregations: sum(_col0)
               minReductionHashAggr: 0.99
               mode: hash
               outputColumnNames: _col0
diff --git a/ql/src/test/results/clientpositive/auto_join21.q.out b/ql/src/test/results/clientpositive/auto_join21.q.out
index b0af07c..35a0f32 100644
--- a/ql/src/test/results/clientpositive/auto_join21.q.out
+++ b/ql/src/test/results/clientpositive/auto_join21.q.out
@@ -22,49 +22,53 @@ STAGE PLANS:
   Stage: Stage-10
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2 
+        $hdt$_1:$hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2 
+        $hdt$_1:$hdt$_1:src1 
           TableScan
-            alias: src2
-            filterExpr: (key > 10) (type: boolean)
+            alias: src1
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: (key > 10) (type: boolean)
-              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 {(key < 10)}
-                  1 
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-8
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: src1
+            alias: src2
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Left Outer Join 0 to 1
-              filter predicates:
-                0 {(key < 10)}
-                1 
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 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
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 356 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -84,10 +88,10 @@ STAGE PLANS:
             HashTable Sink Operator
               filter predicates:
                 0 
-                1 {(key < 10)}
+                1 {_col2}
               keys:
-                0 _col5 (type: string)
-                1 key (type: string)
+                0 _col2 (type: string)
+                1 _col0 (type: string)
 
   Stage: Stage-6
     Map Reduce
@@ -95,21 +99,21 @@ STAGE PLANS:
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key < 10)}
-              keys:
-                0 _col5 (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {_col2}
+                keys:
+                  0 _col2 (type: string)
+                  1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -128,16 +132,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -148,44 +152,44 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col5 (type: string)
+              key expressions: _col2 (type: string)
               null sort order: z
               sort order: +
-              Map-reduce partition columns: _col5 (type: string)
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+              Map-reduce partition columns: _col2 (type: string)
+              Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 10)}
+            1 {VALUE._col1}
           keys:
-            0 _col5 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 1705 Data size: 687326 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
+            0 _col2 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 501 Data size: 89890 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-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/auto_join23.q.out b/ql/src/test/results/clientpositive/auto_join23.q.out
index 9dcfc1a..e91161a 100644
--- a/ql/src/test/results/clientpositive/auto_join23.q.out
+++ b/ql/src/test/results/clientpositive/auto_join23.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[14][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain 
 SELECT  *  FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
 PREHOOK: type: QUERY
@@ -18,43 +18,47 @@ STAGE PLANS:
   Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_0:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_0:src1 
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 
-                  1 
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 
+                    1 
 
   Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 
-                  1 
-                outputColumnNames: _col0, _col1, _col5, _col6
-                Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 
+                    1 
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
@@ -84,7 +88,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[14][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT  *  FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
diff --git a/ql/src/test/results/clientpositive/auto_join28.q.out b/ql/src/test/results/clientpositive/auto_join28.q.out
index 6b27398..e64539e 100644
--- a/ql/src/test/results/clientpositive/auto_join28.q.out
+++ b/ql/src/test/results/clientpositive/auto_join28.q.out
@@ -22,49 +22,53 @@ STAGE PLANS:
   Stage: Stage-10
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2 
+        $hdt$_1:$hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2 
+        $hdt$_1:$hdt$_1:src1 
           TableScan
-            alias: src2
-            filterExpr: (key > 10) (type: boolean)
+            alias: src1
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: (key > 10) (type: boolean)
-              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 {(key < 10)}
-                  1 
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-8
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: src1
+            alias: src2
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Left Outer Join 0 to 1
-              filter predicates:
-                0 {(key < 10)}
-                1 
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 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
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 356 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -84,10 +88,10 @@ STAGE PLANS:
             HashTable Sink Operator
               filter predicates:
                 0 
-                1 {(key < 10)}
+                1 {_col2}
               keys:
-                0 _col5 (type: string)
-                1 key (type: string)
+                0 _col2 (type: string)
+                1 _col0 (type: string)
 
   Stage: Stage-6
     Map Reduce
@@ -95,21 +99,21 @@ STAGE PLANS:
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key < 10)}
-              keys:
-                0 _col5 (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {_col2}
+                keys:
+                  0 _col2 (type: string)
+                  1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -128,16 +132,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -148,44 +152,44 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col5 (type: string)
+              key expressions: _col2 (type: string)
               null sort order: z
               sort order: +
-              Map-reduce partition columns: _col5 (type: string)
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+              Map-reduce partition columns: _col2 (type: string)
+              Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 10)}
+            1 {VALUE._col1}
           keys:
-            0 _col5 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 1705 Data size: 687326 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
+            0 _col2 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 501 Data size: 89890 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-0
     Fetch Operator
@@ -212,40 +216,48 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2 
+        $hdt$_1:src2 
           Fetch Operator
             limit: -1
-        src3 
+        $hdt$_2:src3 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2 
+        $hdt$_1:src2 
           TableScan
             alias: src2
-            filterExpr: (key > 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) > 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key > 10) (type: boolean)
+              predicate: (UDFToDouble(key) > 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 {(key < 10)}
-                  1 
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-        src3 
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  filter predicates:
+                    0 {(UDFToDouble(_col0) < 10.0D)}
+                    1 
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+        $hdt$_2:src3 
           TableScan
             alias: src3
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
@@ -253,34 +265,34 @@ STAGE PLANS:
           TableScan
             alias: src1
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Left Outer Join 0 to 1
-              filter predicates:
-                0 {(key < 10)}
-                1 
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
               Map Join Operator
                 condition map:
                      Left Outer Join 0 to 1
+                filter predicates:
+                  0 {(UDFToDouble(_col0) < 10.0D)}
+                  1 
                 keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Left Outer Join 0 to 1
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     null sort order: zzzzzz
                     sort order: ++++++
-                    Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -288,10 +300,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -322,40 +334,48 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_0:src1 
           Fetch Operator
             limit: -1
-        src3 
+        $hdt$_2:src3 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_0:src1 
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 
-                  1 {(key > 10)}
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-        src3 
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  filter predicates:
+                    0 
+                    1 {(UDFToDouble(_col0) > 10.0D)}
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+        $hdt$_2:src3 
           TableScan
             alias: src3
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
@@ -363,34 +383,34 @@ STAGE PLANS:
           TableScan
             alias: src2
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key > 10)}
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
               Map Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {(UDFToDouble(_col0) > 10.0D)}
                 keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Left Outer Join 0 to 1
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     null sort order: zzzzzz
                     sort order: ++++++
-                    Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -398,10 +418,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -437,49 +457,65 @@ STAGE PLANS:
   Stage: Stage-10
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_0:$hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_0:$hdt$_1:src1 
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 
-                  1 {(key > 10)}
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  filter predicates:
+                    0 {_col2}
+                    1 
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-8
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
+            filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key > 10)}
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 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
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string), (UDFToDouble(key) > 10.0D) (type: boolean)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Left Outer Join 0 to 1
+                  filter predicates:
+                    0 {_col2}
+                    1 
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col3, _col4
+                  Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col3 (type: string), _col4 (type: string), _col0 (type: string), _col1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 762 Data size: 182450 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -499,10 +535,10 @@ STAGE PLANS:
             HashTable Sink Operator
               filter predicates:
                 0 
-                1 {(key < 10)}
+                1 {_col2}
               keys:
-                0 _col5 (type: string)
-                1 key (type: string)
+                0 _col2 (type: string)
+                1 _col0 (type: string)
 
   Stage: Stage-6
     Map Reduce
@@ -510,21 +546,21 @@ STAGE PLANS:
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key < 10)}
-              keys:
-                0 _col5 (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-              Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {_col2}
+                keys:
+                  0 _col2 (type: string)
+                  1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -543,16 +579,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -563,44 +599,44 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col5 (type: string)
+              key expressions: _col2 (type: string)
               null sort order: z
               sort order: +
-              Map-reduce partition columns: _col5 (type: string)
+              Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 10)}
+            1 {VALUE._col1}
           keys:
-            0 _col5 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 1705 Data size: 643826 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
+            0 _col2 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 1705 Data size: 687326 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-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/auto_join29.q.out b/ql/src/test/results/clientpositive/auto_join29.q.out
index ade39bd..335b6dc 100644
--- a/ql/src/test/results/clientpositive/auto_join29.q.out
+++ b/ql/src/test/results/clientpositive/auto_join29.q.out
@@ -22,49 +22,53 @@ STAGE PLANS:
   Stage: Stage-10
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2 
+        $hdt$_1:$hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2 
+        $hdt$_1:$hdt$_1:src1 
           TableScan
-            alias: src2
-            filterExpr: (key > 10) (type: boolean)
+            alias: src1
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: (key > 10) (type: boolean)
-              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 {(key < 10)}
-                  1 
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-8
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: src1
+            alias: src2
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Left Outer Join 0 to 1
-              filter predicates:
-                0 {(key < 10)}
-                1 
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 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
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 356 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -84,10 +88,10 @@ STAGE PLANS:
             HashTable Sink Operator
               filter predicates:
                 0 
-                1 {(key < 10)}
+                1 {_col2}
               keys:
-                0 _col5 (type: string)
-                1 key (type: string)
+                0 _col2 (type: string)
+                1 _col0 (type: string)
 
   Stage: Stage-6
     Map Reduce
@@ -95,21 +99,21 @@ STAGE PLANS:
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key < 10)}
-              keys:
-                0 _col5 (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {_col2}
+                keys:
+                  0 _col2 (type: string)
+                  1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -128,16 +132,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -148,44 +152,44 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col5 (type: string)
+              key expressions: _col2 (type: string)
               null sort order: z
               sort order: +
-              Map-reduce partition columns: _col5 (type: string)
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+              Map-reduce partition columns: _col2 (type: string)
+              Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 10)}
+            1 {VALUE._col1}
           keys:
-            0 _col5 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 1705 Data size: 687326 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
+            0 _col2 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 501 Data size: 89890 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-0
     Fetch Operator
@@ -720,40 +724,48 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2 
+        $hdt$_1:src2 
           Fetch Operator
             limit: -1
-        src3 
+        $hdt$_2:src3 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2 
+        $hdt$_1:src2 
           TableScan
             alias: src2
-            filterExpr: (key > 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) > 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key > 10) (type: boolean)
+              predicate: (UDFToDouble(key) > 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 {(key < 10)}
-                  1 
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-        src3 
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  filter predicates:
+                    0 {(UDFToDouble(_col0) < 10.0D)}
+                    1 
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+        $hdt$_2:src3 
           TableScan
             alias: src3
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
@@ -761,34 +773,34 @@ STAGE PLANS:
           TableScan
             alias: src1
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Left Outer Join 0 to 1
-              filter predicates:
-                0 {(key < 10)}
-                1 
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
               Map Join Operator
                 condition map:
                      Left Outer Join 0 to 1
+                filter predicates:
+                  0 {(UDFToDouble(_col0) < 10.0D)}
+                  1 
                 keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Left Outer Join 0 to 1
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     null sort order: zzzzzz
                     sort order: ++++++
-                    Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -796,10 +808,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1338,40 +1350,48 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_0:src1 
           Fetch Operator
             limit: -1
-        src3 
+        $hdt$_2:src3 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_0:src1 
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 
-                  1 {(key > 10)}
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-        src3 
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  filter predicates:
+                    0 
+                    1 {(UDFToDouble(_col0) > 10.0D)}
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+        $hdt$_2:src3 
           TableScan
             alias: src3
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
@@ -1379,34 +1399,34 @@ STAGE PLANS:
           TableScan
             alias: src2
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key > 10)}
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
               Map Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {(UDFToDouble(_col0) > 10.0D)}
                 keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Left Outer Join 0 to 1
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                     null sort order: zzzzzz
                     sort order: ++++++
-                    Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -1414,10 +1434,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1973,49 +1993,65 @@ STAGE PLANS:
   Stage: Stage-10
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_0:$hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_0:$hdt$_1:src1 
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 
-                  1 {(key > 10)}
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  filter predicates:
+                    0 {_col2}
+                    1 
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-8
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
+            filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key > 10)}
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 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
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string), (UDFToDouble(key) > 10.0D) (type: boolean)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Left Outer Join 0 to 1
+                  filter predicates:
+                    0 {_col2}
+                    1 
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col3, _col4
+                  Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col3 (type: string), _col4 (type: string), _col0 (type: string), _col1 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 762 Data size: 182450 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -2035,10 +2071,10 @@ STAGE PLANS:
             HashTable Sink Operator
               filter predicates:
                 0 
-                1 {(key < 10)}
+                1 {_col2}
               keys:
-                0 _col5 (type: string)
-                1 key (type: string)
+                0 _col2 (type: string)
+                1 _col0 (type: string)
 
   Stage: Stage-6
     Map Reduce
@@ -2046,21 +2082,21 @@ STAGE PLANS:
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key < 10)}
-              keys:
-                0 _col5 (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-              Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {_col2}
+                keys:
+                  0 _col2 (type: string)
+                  1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -2079,16 +2115,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2099,44 +2135,44 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col5 (type: string)
+              key expressions: _col2 (type: string)
               null sort order: z
               sort order: +
-              Map-reduce partition columns: _col5 (type: string)
+              Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 10)}
+            1 {VALUE._col1}
           keys:
-            0 _col5 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 1705 Data size: 643826 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
+            0 _col2 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 1705 Data size: 687326 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-0
     Fetch Operator
@@ -2683,73 +2719,79 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_0:$hdt$_0:src1 
           Fetch Operator
             limit: -1
-        src3 
+        $hdt$_2:src3 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_0:$hdt$_0:src1 
           TableScan
             alias: src1
-            filterExpr: ((key > 10) and (key < 10)) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: ((key > 10) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-        src3 
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+        $hdt$_2:src3 
           TableScan
             alias: src3
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
-            filterExpr: ((key < 10) and (key > 10)) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: ((key < 10) and (key > 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6
-                Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                 Map Join Operator
                   condition map:
-                       Left Outer Join 0 to 1
+                       Inner Join 0 to 1
                   keys:
-                    0 _col5 (type: string)
-                    1 key (type: string)
-                  outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                  Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map Join Operator
+                    condition map:
+                         Left Outer Join 0 to 1
+                    keys:
+                      0 _col2 (type: string)
+                      1 _col0 (type: string)
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                    Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                       null sort order: zzzzzz
                       sort order: ++++++
-                      Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -2757,10 +2799,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 2 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 2 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2804,47 +2846,56 @@ STAGE PLANS:
   Stage: Stage-10
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_1:$hdt$_1:src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_1:$hdt$_1:src1 
           TableScan
             alias: src1
-            filterExpr: ((key > 10) and (key < 10)) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: ((key > 10) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-8
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
-            filterExpr: ((key < 10) and (key > 10)) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: ((key < 10) and (key > 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6
-                Statistics: Num rows: 86 Data size: 30616 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
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map Join Operator
+                    condition map:
+                         Inner Join 0 to 1
+                    keys:
+                      0 _col0 (type: string)
+                      1 _col0 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 1 Data size: 356 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
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -2864,10 +2915,10 @@ STAGE PLANS:
             HashTable Sink Operator
               filter predicates:
                 0 
-                1 {(key < 10)}
+                1 {_col2}
               keys:
-                0 _col5 (type: string)
-                1 key (type: string)
+                0 _col2 (type: string)
+                1 _col0 (type: string)
 
   Stage: Stage-6
     Map Reduce
@@ -2875,21 +2926,21 @@ STAGE PLANS:
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Right Outer Join 0 to 1
-              filter predicates:
-                0 
-                1 {(key < 10)}
-              keys:
-                0 _col5 (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-              Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Map Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {_col2}
+                keys:
+                  0 _col2 (type: string)
+                  1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
                   table:
@@ -2908,16 +2959,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2928,44 +2979,44 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col5 (type: string)
+              key expressions: _col2 (type: string)
               null sort order: z
               sort order: +
-              Map-reduce partition columns: _col5 (type: string)
-              Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+              Map-reduce partition columns: _col2 (type: string)
+              Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 10)}
+            1 {VALUE._col1}
           keys:
-            0 _col5 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 636 Data size: 161980 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
+            0 _col2 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 501 Data size: 89890 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-0
     Fetch Operator
@@ -3500,78 +3551,82 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2 
+        $hdt$_0:$hdt$_0:src1 
           Fetch Operator
             limit: -1
-        src3 
+        $hdt$_2:src3 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2 
+        $hdt$_0:$hdt$_0:src1 
           TableScan
-            alias: src2
-            filterExpr: (key > 10) (type: boolean)
+            alias: src1
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: (key > 10) (type: boolean)
-              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 {(key < 10)}
-                  1 
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-        src3 
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+        $hdt$_2:src3 
           TableScan
             alias: src3
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: src1
+            alias: src2
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Map Join Operator
-              condition map:
-                   Left Outer Join 0 to 1
-              filter predicates:
-                0 {(key < 10)}
-                1 
-              keys:
-                0 key (type: string)
-                1 key (type: string)
-              outputColumnNames: _col0, _col1, _col5, _col6
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-              Filter Operator
-                predicate: (_col5 < 10) (type: boolean)
-                Statistics: Num rows: 254 Data size: 60876 Basic stats: COMPLETE Column stats: COMPLETE
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 _col5 (type: string)
-                    1 key (type: string)
-                  outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                  Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                    Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
-                      null sort order: zzzzzz
-                      sort order: ++++++
-                      Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: (UDFToDouble(_col0) < 10.0D) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map Join Operator
+                    condition map:
+                         Inner Join 0 to 1
+                    keys:
+                      0 _col0 (type: string)
+                      1 _col0 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 _col2 (type: string)
+                        1 _col0 (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                      Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                        null sort order: zzzzzz
+                        sort order: ++++++
+                        Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -3579,10 +3634,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3602,6 +3657,7 @@ POSTHOOK: query: SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key =
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 #### A masked pattern was here ####
+Warning: Map Join MAPJOIN[27][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
 PREHOOK: query: explain
 SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value
 PREHOOK: type: QUERY
@@ -3621,79 +3677,86 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_0:$hdt$_0:src1 
           Fetch Operator
             limit: -1
-        src3 
+        $hdt$_2:src3 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_0:$hdt$_0:src1 
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: (key < 10) (type: boolean)
-              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                filter predicates:
-                  0 
-                  1 {(key > 10)}
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-        src3 
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  filter predicates:
+                    0 
+                    1 {true}
+                  keys:
+                    0 
+                    1 
+        $hdt$_2:src3 
           TableScan
             alias: src3
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Right Outer Join 0 to 1
-                filter predicates:
-                  0 
-                  1 {(key > 10)}
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6
-                Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                 Map Join Operator
                   condition map:
-                       Inner Join 0 to 1
+                       Right Outer Join 0 to 1
+                  filter predicates:
+                    0 
+                    1 {true}
                   keys:
-                    0 _col5 (type: string)
-                    1 key (type: string)
-                  outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                  Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                    0 
+                    1 
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map Join Operator
+                    condition map:
+                         Inner Join 0 to 1
+                    keys:
+                      0 _col2 (type: string)
+                      1 _col0 (type: string)
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                    Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 166 Data size: 88644 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                       null sort order: zzzzzz
                       sort order: ++++++
-                      Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 166 Data size: 88644 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -3701,10 +3764,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 166 Data size: 88644 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 166 Data size: 88644 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3716,6 +3779,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+Warning: Map Join MAPJOIN[27][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
 PREHOOK: query: SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -3765,73 +3829,82 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src1 
+        $hdt$_0:$hdt$_0:src1 
           Fetch Operator
             limit: -1
-        src3 
+        $hdt$_2:src3 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src1 
+        $hdt$_0:$hdt$_0:src1 
           TableScan
             alias: src1
-            filterExpr: ((key > 10) and (key < 10)) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: ((key > 10) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-        src3 
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+        $hdt$_2:src3 
           TableScan
             alias: src3
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: string)
-                  1 key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src2
-            filterExpr: ((key < 10) and (key > 10)) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: ((key < 10) and (key > 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 key (type: string)
-                  1 key (type: string)
-                outputColumnNames: _col0, _col1, _col5, _col6
-                Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 _col5 (type: string)
-                    1 key (type: string)
-                  outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                  Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                    Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
-                      null sort order: zzzzzz
-                      sort order: ++++++
-                      Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: (UDFToDouble(_col0) < 10.0D) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map Join Operator
+                    condition map:
+                         Inner Join 0 to 1
+                    keys:
+                      0 _col0 (type: string)
+                      1 _col0 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 _col2 (type: string)
+                        1 _col0 (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                      Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                        null sort order: zzzzzz
+                        sort order: ++++++
+                        Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
@@ -3839,10 +3912,10 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/auto_join31.q.out b/ql/src/test/results/clientpositive/auto_join31.q.out
index 73651d3..0c0e474 100644
--- a/ql/src/test/results/clientpositive/auto_join31.q.out
+++ b/ql/src/test/results/clientpositive/auto_join31.q.out
@@ -33,26 +33,14 @@ STAGE PLANS:
   Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        x:src 
+        $hdt$_0:$hdt$_0:src 
           Fetch Operator
             limit: -1
-        z:src 
+        $hdt$_0:$hdt$_2:src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        x:src 
-          TableScan
-            alias: src
-            Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string)
-              outputColumnNames: _col0
-              Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col0 (type: string)
-                  1 _col0 (type: string)
-        z:src 
+        $hdt$_0:$hdt$_0:src 
           TableScan
             alias: src
             filterExpr: key is not null (type: boolean)
@@ -68,28 +56,45 @@ STAGE PLANS:
                   keys:
                     0 _col0 (type: string)
                     1 _col0 (type: string)
+        $hdt$_0:$hdt$_2:src 
+          TableScan
+            alias: src
+            filterExpr: key is not null (type: boolean)
+            Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
 
   Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              Map Join Operator
-                condition map:
-                     Right Outer Join 0 to 1
-                keys:
-                  0 _col0 (type: string)
-                  1 _col0 (type: string)
-                outputColumnNames: _col0, _col2, _col3
-                Statistics: Num rows: 791 Data size: 209615 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: _col0 is not null (type: boolean)
-                  Statistics: Num rows: 791 Data size: 209615 Basic stats: COMPLETE Column stats: COMPLETE
+            filterExpr: key is not null (type: boolean)
+            Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  keys:
+                    0 _col0 (type: string)
+                    1 _col0 (type: string)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 791 Data size: 68817 Basic stats: COMPLETE Column stats: COMPLETE
                   Map Join Operator
                     condition map:
                          Inner Join 0 to 1
@@ -98,17 +103,21 @@ STAGE PLANS:
                       1 _col0 (type: string)
                     outputColumnNames: _col2, _col3
                     Statistics: Num rows: 1251 Data size: 222678 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: sum(hash(_col2,_col3))
-                      minReductionHashAggr: 0.99
-                      mode: hash
+                    Select Operator
+                      expressions: hash(_col2,_col3) (type: int)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        null sort order: 
-                        sort order: 
+                      Statistics: Num rows: 1251 Data size: 222678 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: sum(_col0)
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: bigint)
+                        Reduce Output Operator
+                          null sort order: 
+                          sort order: 
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: bigint)
       Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
index 72e041e..0035b52 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
@@ -1,4 +1,4 @@
-Warning: Map Join MAPJOIN[20][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[18][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain 
 select sum(hash(a.k1,a.v1,a.k2, a.v2))
 from (
@@ -40,15 +40,15 @@ STAGE PLANS:
       Alias -> Map Local Operator Tree:
         a:cbo_t1:cbo_t3 
           TableScan
-            alias: cbo_t3
-            filterExpr: (key < 10) (type: boolean)
+            alias: a:cbo_t1:cbo_t3
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 20 Data size: 3230 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 6 Data size: 1020 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 6 Data size: 1020 Basic stats: COMPLETE Column stats: COMPLETE
                 HashTable Sink Operator
                   keys:
@@ -59,15 +59,15 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: cbo_t3
-            filterExpr: (key < 10) (type: boolean)
+            alias: a:cbo_t2:cbo_t3
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 20 Data size: 3230 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 6 Data size: 1020 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 6 Data size: 1020 Basic stats: COMPLETE Column stats: COMPLETE
                 Map Join Operator
                   condition map:
@@ -75,14 +75,14 @@ STAGE PLANS:
                   keys:
                     0 
                     1 
-                  outputColumnNames: _col0, _col1, _col5, _col6
+                  outputColumnNames: key, value, key0, value0
                   Statistics: Num rows: 36 Data size: 12240 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string)
-                    outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 36 Data size: 12240 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: hash(key,value,key0,value0) (type: int)
+                    outputColumnNames: $f0
+                    Statistics: Num rows: 36 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: sum(hash(_col0,_col1,_col2,_col3))
+                      aggregations: sum($f0)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
@@ -99,19 +99,15 @@ STAGE PLANS:
         Group By Operator
           aggregations: sum(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
+          outputColumnNames: $f0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: bigint)
-            outputColumnNames: _col0
+          File Output Operator
+            compressed: false
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-              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
+            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
@@ -119,7 +115,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[20][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[18][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain 
 select sum(hash(a.k1,a.v1,a.k2, a.v2))
 from (
@@ -161,15 +157,15 @@ STAGE PLANS:
       Alias -> Map Local Operator Tree:
         a:cbo_t1:cbo_t3 
           TableScan
-            alias: cbo_t3
-            filterExpr: (key < 10) (type: boolean)
+            alias: a:cbo_t1:cbo_t3
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 20 Data size: 3230 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 6 Data size: 1020 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 6 Data size: 1020 Basic stats: COMPLETE Column stats: COMPLETE
                 HashTable Sink Operator
                   keys:
@@ -180,15 +176,15 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: cbo_t3
-            filterExpr: (key < 10) (type: boolean)
+            alias: a:cbo_t2:cbo_t3
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 20 Data size: 3230 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 6 Data size: 1020 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 6 Data size: 1020 Basic stats: COMPLETE Column stats: COMPLETE
                 Map Join Operator
                   condition map:
@@ -196,14 +192,14 @@ STAGE PLANS:
                   keys:
                     0 
                     1 
-                  outputColumnNames: _col0, _col1, _col5, _col6
+                  outputColumnNames: key, value, key0, value0
                   Statistics: Num rows: 36 Data size: 12240 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string)
-                    outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 36 Data size: 12240 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: hash(key,value,key0,value0) (type: int)
+                    outputColumnNames: $f0
+                    Statistics: Num rows: 36 Data size: 144 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: sum(hash(_col0,_col1,_col2,_col3))
+                      aggregations: sum($f0)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
@@ -220,19 +216,15 @@ STAGE PLANS:
         Group By Operator
           aggregations: sum(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
+          outputColumnNames: $f0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: bigint)
-            outputColumnNames: _col0
+          File Output Operator
+            compressed: false
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-              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
+            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
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer14.q.out b/ql/src/test/results/clientpositive/correlationoptimizer14.q.out
index b8d7648..2962ea9 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer14.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer14.q.out
@@ -169,33 +169,32 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: x
-            filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: key is not null (type: boolean)
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
                 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: string)
-                  null sort order: z
-                  sort order: +
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: string)
+                value expressions: _col1 (type: string)
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 500 Data size: 89000 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
+          Filter Operator
+            predicate: _col0 is not null (type: boolean)
+            Statistics: Num rows: 500 Data size: 89000 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
@@ -238,33 +237,32 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: y
-            filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: key is not null (type: boolean)
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
               Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-              Select Operator
-                expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
                 Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col0 (type: string)
-                  null sort order: z
-                  sort order: +
-                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: string)
+                value expressions: _col1 (type: string)
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 25 Data size: 4375 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
+          Filter Operator
+            predicate: _col0 is not null (type: boolean)
+            Statistics: Num rows: 25 Data size: 4375 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-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/identity_project_remove_skip.q.out b/ql/src/test/results/clientpositive/identity_project_remove_skip.q.out
index 1176c7c..756309c 100644
--- a/ql/src/test/results/clientpositive/identity_project_remove_skip.q.out
+++ b/ql/src/test/results/clientpositive/identity_project_remove_skip.q.out
@@ -1,3 +1,4 @@
+Warning: Map Join MAPJOIN[13][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
 PREHOOK: query: explain
 select t2.* 
 from
@@ -29,11 +30,11 @@ STAGE PLANS:
   Stage: Stage-4
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        t2:t1:src 
+        $hdt$_0:src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        t2:t1:src 
+        $hdt$_0:src 
           TableScan
             alias: src
             filterExpr: ((value = 'val_105') and (key = '105')) (type: boolean)
@@ -42,11 +43,11 @@ STAGE PLANS:
               predicate: ((value = 'val_105') and (key = '105')) (type: boolean)
               Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                 HashTable Sink Operator
                   keys:
-                    0 '105' (type: string)
-                    1 '105' (type: string)
+                    0 
+                    1 
 
   Stage: Stage-3
     Map Reduce
@@ -64,16 +65,16 @@ STAGE PLANS:
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 '105' (type: string)
-                    1 '105' (type: string)
+                    0 
+                    1 
                   Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: '105' (type: string), 'val_105' (type: string)
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
-                      Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -88,6 +89,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+Warning: Map Join MAPJOIN[13][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
 PREHOOK: query: select t2.* 
 from
   (select key,value from (select key,value from src) t1 sort by key) t2
diff --git a/ql/src/test/results/clientpositive/input_part7.q.out b/ql/src/test/results/clientpositive/input_part7.q.out
index 15a388e..5ac50a4 100644
--- a/ql/src/test/results/clientpositive/input_part7.q.out
+++ b/ql/src/test/results/clientpositive/input_part7.q.out
@@ -32,54 +32,54 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: x
-            filterExpr: ((ds = '2008-04-08') and (key < 100)) (type: boolean)
+            filterExpr: ((UDFToDouble(key) < 100.0D) and (ds = '2008-04-08')) (type: boolean)
             Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (key < 100) (type: boolean)
+              predicate: (UDFToDouble(key) < 100.0D) (type: boolean)
               Statistics: Num rows: 333 Data size: 120546 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string), hr (type: string)
-                outputColumnNames: _col0, _col1, _col3
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 333 Data size: 120546 Basic stats: COMPLETE Column stats: COMPLETE
                 Union
                   Statistics: Num rows: 666 Data size: 241092 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+                    expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                     outputColumnNames: _col0, _col1, _col3
-                    Statistics: Num rows: 666 Data size: 241092 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string)
-                      null sort order: zzzz
-                      sort order: ++++
-                      Statistics: Num rows: 666 Data size: 241092 Basic stats: COMPLETE Column stats: COMPLETE
+                      key expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+                      null sort order: zzz
+                      sort order: +++
+                      Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                       tag: -1
                       auto parallelism: false
           TableScan
             alias: y
-            filterExpr: ((ds = '2008-04-08') and (key < 100)) (type: boolean)
+            filterExpr: ((UDFToDouble(key) < 100.0D) and (ds = '2008-04-08')) (type: boolean)
             Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (key < 100) (type: boolean)
+              predicate: (UDFToDouble(key) < 100.0D) (type: boolean)
               Statistics: Num rows: 333 Data size: 120546 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string), hr (type: string)
-                outputColumnNames: _col0, _col1, _col3
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 333 Data size: 120546 Basic stats: COMPLETE Column stats: COMPLETE
                 Union
                   Statistics: Num rows: 666 Data size: 241092 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+                    expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                     outputColumnNames: _col0, _col1, _col3
-                    Statistics: Num rows: 666 Data size: 241092 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      key expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string)
-                      null sort order: zzzz
-                      sort order: ++++
-                      Statistics: Num rows: 666 Data size: 241092 Basic stats: COMPLETE Column stats: COMPLETE
+                      key expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
+                      null sort order: zzz
+                      sort order: +++
+                      Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                       tag: -1
                       auto parallelism: false
       Path -> Alias:
@@ -184,12 +184,12 @@ STAGE PLANS:
               name: default.srcpart
             name: default.srcpart
       Truncated Path -> Alias:
-        /srcpart/ds=2008-04-08/hr=11 [a:__u1-subquery1:x, a:__u1-subquery2:y]
-        /srcpart/ds=2008-04-08/hr=12 [a:__u1-subquery1:x, a:__u1-subquery2:y]
+        /srcpart/ds=2008-04-08/hr=11 [$hdt$_0-subquery1:x, $hdt$_0-subquery2:y]
+        /srcpart/ds=2008-04-08/hr=12 [$hdt$_0-subquery1:x, $hdt$_0-subquery2:y]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), '2008-04-08' (type: string), KEY.reducesinkkey3 (type: string)
+          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), '2008-04-08' (type: string), KEY.reducesinkkey2 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
diff --git a/ql/src/test/results/clientpositive/join0.q.out b/ql/src/test/results/clientpositive/join0.q.out
index cda0f3f..9f2e181 100644
--- a/ql/src/test/results/clientpositive/join0.q.out
+++ b/ql/src/test/results/clientpositive/join0.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 
@@ -30,10 +30,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string)
@@ -46,10 +46,10 @@ STAGE PLANS:
                   value expressions: _col0 (type: string), _col1 (type: string)
           TableScan
             alias: src
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: string), value (type: string)
@@ -104,7 +104,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN FORMATTED
 SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 
@@ -125,8 +125,8 @@ SELECT src1.key as k1, src1.value as v1,
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 #### A masked pattern was here ####
-{"cboInfo":"Plan not optimized by CBO because the statement has sort by","STAGE DEPENDENCIES":{"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-0":{"DEPENDENT STAGES":"Stage-2"}},"STAGE PLANS":{"Stage-1":{"Map Reduce":{"Map Operator Tree:":[{"TableScan":{"alias:":"src","columns:":["key","value"],"database:":"default","filterExpr:":"(key < 10) (type: boolean)","Statistics:":"Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE","table: [...]
-Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
+{"CBOPlan":"{\n  \"rels\": [\n    {\n      \"id\": \"0\",\n      \"relOp\": \"org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan\",\n      \"table\": [\n        \"default\",\n        \"src\"\n      ],\n      \"table:alias\": \"src\",\n      \"inputs\": [],\n      \"rowCount\": 500.0,\n      \"avgRowSize\": 9.624,\n      \"rowType\": [\n        {\n          \"type\": \"VARCHAR\",\n          \"nullable\": true,\n          \"precision\": 2147483647,\n          \"name\":  [...]
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 
   (SELECT * FROM src WHERE src.key < 10) src1 
diff --git a/ql/src/test/results/clientpositive/join15.q.out b/ql/src/test/results/clientpositive/join15.q.out
index 664f2e0..bbde586 100644
--- a/ql/src/test/results/clientpositive/join15.q.out
+++ b/ql/src/test/results/clientpositive/join15.q.out
@@ -24,13 +24,17 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
           TableScan
             alias: src2
             filterExpr: key is not null (type: boolean)
@@ -38,32 +42,32 @@ STAGE PLANS:
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 key (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 791 Data size: 281596 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 791 Data size: 281596 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
+          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
diff --git a/ql/src/test/results/clientpositive/join20.q.out b/ql/src/test/results/clientpositive/join20.q.out
index 9327cf4..02f71dd 100644
--- a/ql/src/test/results/clientpositive/join20.q.out
+++ b/ql/src/test/results/clientpositive/join20.q.out
@@ -22,41 +22,49 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
           TableScan
             alias: src2
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 key (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6
-          Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -73,40 +81,40 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 20.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 20)}
+            1 {VALUE._col1}
           keys:
             0 _col0 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 914 Data size: 310432 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
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 762 Data size: 229264 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-3
     Map Reduce
@@ -116,16 +124,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -719,41 +727,49 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src1
-            filterExpr: ((key < 15) and (key < 10)) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: ((key < 15) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
-                Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
           TableScan
             alias: src2
-            filterExpr: ((key < 10) and (key < 15)) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: ((key < 10) and (key < 15)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
-                Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 key (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6
-          Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -770,40 +786,40 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 20.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 20)}
+            1 {VALUE._col1}
           keys:
             0 _col0 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 636 Data size: 161980 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
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 762 Data size: 229264 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-3
     Map Reduce
@@ -813,16 +829,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/join21.q.out b/ql/src/test/results/clientpositive/join21.q.out
index d25eb5c..5866357 100644
--- a/ql/src/test/results/clientpositive/join21.q.out
+++ b/ql/src/test/results/clientpositive/join21.q.out
@@ -21,39 +21,46 @@ STAGE PLANS:
           TableScan
             alias: src1
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
           TableScan
             alias: src2
-            filterExpr: (key > 10) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Filter Operator
-              predicate: (key > 10) (type: boolean)
-              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
-                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+              Limit
+                Number of rows: 0
+                Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
-               Left Outer Join 0 to 1
-          filter predicates:
-            0 {(KEY.reducesinkkey0 < 10)}
-            1 
+               Inner Join 0 to 1
           keys:
-            0 key (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6
-          Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -66,44 +73,44 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col5 (type: string)
+              key expressions: _col2 (type: string)
               null sort order: z
               sort order: +
-              Map-reduce partition columns: _col5 (type: string)
-              Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+              Map-reduce partition columns: _col2 (type: string)
+              Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 10)}
+            1 {VALUE._col1}
           keys:
-            0 _col5 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 1705 Data size: 687326 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
+            0 _col2 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 501 Data size: 89890 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-3
     Map Reduce
@@ -113,16 +120,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/join23.q.out b/ql/src/test/results/clientpositive/join23.q.out
index 4f8f0d8..23b4207 100644
--- a/ql/src/test/results/clientpositive/join23.q.out
+++ b/ql/src/test/results/clientpositive/join23.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join JOIN[4][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *  FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
 PREHOOK: type: QUERY
@@ -20,28 +20,36 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                null sort order: 
-                sort order: 
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: key (type: string), value (type: string)
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string)
           TableScan
             alias: src2
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                null sort order: 
-                sort order: 
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: key (type: string), value (type: string)
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -49,18 +57,14 @@ STAGE PLANS:
           keys:
             0 
             1 
-          outputColumnNames: _col0, _col1, _col5, _col6
+          outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 27556 Data size: 9809936 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
+          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
@@ -91,7 +95,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[4][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT *  FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
diff --git a/ql/src/test/results/clientpositive/join40.q.out b/ql/src/test/results/clientpositive/join40.q.out
index b0942ea..f33ff5a 100644
--- a/ql/src/test/results/clientpositive/join40.q.out
+++ b/ql/src/test/results/clientpositive/join40.q.out
@@ -1789,41 +1789,49 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src1
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
           TableScan
             alias: src2
-            filterExpr: (key < 10) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (key < 10) (type: boolean)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
               Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 key (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6
-          Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -1840,40 +1848,40 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 20.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 20)}
+            1 {VALUE._col1}
           keys:
             0 _col0 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 914 Data size: 310432 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
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 762 Data size: 229264 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-3
     Map Reduce
@@ -1883,16 +1891,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 914 Data size: 310432 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2486,41 +2494,49 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src1
-            filterExpr: ((key < 15) and (key < 10)) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: ((key < 15) and (key < 10)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
-                Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
           TableScan
             alias: src2
-            filterExpr: ((key < 10) and (key < 15)) (type: boolean)
+            filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: ((key < 10) and (key < 15)) (type: boolean)
-              Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                null sort order: z
-                sort order: +
-                Map-reduce partition columns: key (type: string)
-                Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: value (type: string)
+              predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+              Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 key (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6
-          Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
+            0 _col0 (type: string)
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -2537,40 +2553,40 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
-              Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: _col1 (type: string), _col5 (type: string), _col6 (type: string)
+              Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
+              value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
           TableScan
             alias: src3
             Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-            Reduce Output Operator
-              key expressions: key (type: string)
-              null sort order: z
-              sort order: +
-              Map-reduce partition columns: key (type: string)
-              Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-              value expressions: value (type: string)
+            Select Operator
+              expressions: key (type: string), value (type: string), (UDFToDouble(key) < 20.0D) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+              Reduce Output Operator
+                key expressions: _col0 (type: string)
+                null sort order: z
+                sort order: +
+                Map-reduce partition columns: _col0 (type: string)
+                Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                value expressions: _col1 (type: string), _col2 (type: boolean)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Right Outer Join 0 to 1
           filter predicates:
             0 
-            1 {(KEY.reducesinkkey0 < 20)}
+            1 {VALUE._col1}
           keys:
             0 _col0 (type: string)
-            1 key (type: string)
-          outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-          Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
-          Select Operator
-            expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 636 Data size: 161980 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
+            1 _col0 (type: string)
+          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+          Statistics: Num rows: 762 Data size: 229264 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-3
     Map Reduce
@@ -2580,16 +2596,16 @@ STAGE PLANS:
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
               null sort order: zzzzzz
               sort order: ++++++
-              Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 762 Data size: 229264 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
index 699398b..aeeffa5 100644
--- a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
@@ -137,21 +137,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcpart_acid
-                  filterExpr: (((UDFToInteger(key) = 413) or (UDFToInteger(key) = 43)) and (hr = '11')) (type: boolean)
+                  filterExpr: ((UDFToInteger(key)) IN (413, 43) and (hr = '11')) (type: boolean)
                   Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: PARTIAL
                   Filter Operator
-                    predicate: ((UDFToInteger(key) = 413) or (UDFToInteger(key) = 43)) (type: boolean)
-                    Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: PARTIAL
+                    predicate: (UDFToInteger(key)) IN (413, 43) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 181000 Basic stats: COMPLETE Column stats: PARTIAL
                     Select Operator
                       expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: string), concat(value, 'updated') (type: string), ds (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1000 Data size: 617000 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
                       Reduce Output Operator
                         key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
                         null sort order: z
                         sort order: +
                         Map-reduce partition columns: UDFToInteger(_col0) (type: int)
-                        Statistics: Num rows: 1000 Data size: 617000 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
                         value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
             Execution mode: llap
             LLAP IO: may be used (ACID table)
@@ -161,10 +161,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), '11' (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 1000 Data size: 617000 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1000 Data size: 617000 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
                   table:
                       input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                       output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
@@ -812,21 +812,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcpart_acidb
-                  filterExpr: (((UDFToInteger(key) = 413) or (UDFToInteger(key) = 43)) and (hr = '11')) (type: boolean)
+                  filterExpr: ((UDFToInteger(key)) IN (413, 43) and (hr = '11')) (type: boolean)
                   Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: PARTIAL
                   Filter Operator
-                    predicate: ((UDFToInteger(key) = 413) or (UDFToInteger(key) = 43)) (type: boolean)
-                    Statistics: Num rows: 1000 Data size: 362000 Basic stats: COMPLETE Column stats: PARTIAL
+                    predicate: (UDFToInteger(key)) IN (413, 43) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 181000 Basic stats: COMPLETE Column stats: PARTIAL
                     Select Operator
                       expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), key (type: string), concat(value, 'updated') (type: string), ds (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1000 Data size: 617000 Basic stats: COMPLETE Column stats: PARTIAL
+                      Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
                       Reduce Output Operator
                         key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
                         null sort order: z
                         sort order: +
                         Map-reduce partition columns: UDFToInteger(_col0) (type: int)
-                        Statistics: Num rows: 1000 Data size: 617000 Basic stats: COMPLETE Column stats: PARTIAL
+                        Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
                         value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
             Execution mode: llap
             LLAP IO: may be used (ACID table)
@@ -836,10 +836,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), '11' (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Statistics: Num rows: 1000 Data size: 617000 Basic stats: COMPLETE Column stats: PARTIAL
+                Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1000 Data size: 617000 Basic stats: COMPLETE Column stats: PARTIAL
+                  Statistics: Num rows: 500 Data size: 308500 Basic stats: COMPLETE Column stats: PARTIAL
                   table:
                       input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                       output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
@@ -1371,19 +1371,19 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprOrExpr(children: FilterLongColEqualLongScalar(col 5:int, val 413)(children: CastStringToLong(col 0:string) -> 5:int), FilterLongColEqualLongScalar(col 6:int, val 43)(children: CastStringToLong(col 0:string) -> 6:int))
+                        predicateExpression: FilterLongColumnInList(col 5:int, values [413, 43])(children: CastStringToLong(col 0:string) -> 5:int)
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                          projectedOutputColumnNums: [4, 0, 7, 2]
-                          selectExpressions: StringGroupColConcatStringScalar(col 1:string, val updated) -> 7:string
+                          projectedOutputColumnNums: [4, 0, 6, 2]
+                          selectExpressions: StringGroupColConcatStringScalar(col 1:string, val updated) -> 6:string
                         Reduce Sink Vectorization:
                             className: VectorReduceSinkObjectHashOperator
                             keyColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
                             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
                             partitionColumns: 5:int
-                            valueColumns: 0:string, 7:string, 2:string
+                            valueColumns: 0:string, 6:string, 2:string
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
             Map Vectorization:
@@ -1402,7 +1402,7 @@ STAGE PLANS:
                     neededVirtualColumns: [ROWID]
                     partitionColumnCount: 2
                     partitionColumns: ds:string, hr:string
-                    scratchColumnTypeNames: [bigint, bigint, string]
+                    scratchColumnTypeNames: [bigint, string]
         Reducer 2 
             Execution mode: vectorized, llap
             Reduce Vectorization:
@@ -1758,45 +1758,43 @@ STAGE DEPENDENCIES:
   Stage-5 depends on stages: Stage-4
   Stage-0 depends on stages: Stage-5
   Stage-6 depends on stages: Stage-0
+  Stage-1 depends on stages: Stage-5
+  Stage-7 depends on stages: Stage-1
   Stage-2 depends on stages: Stage-5
-  Stage-7 depends on stages: Stage-2
+  Stage-8 depends on stages: Stage-2
   Stage-3 depends on stages: Stage-5
-  Stage-8 depends on stages: Stage-3
-  Stage-1 depends on stages: Stage-5
-  Stage-9 depends on stages: Stage-1
+  Stage-9 depends on stages: Stage-3
 
 STAGE PLANS:
   Stage: Stage-4
     Tez
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 8 <- Map 7 (SIMPLE_EDGE)
       Vertices:
         Map 1 
             Map Operator Tree:
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:key:string, 1:value:string, 2:ds:string, 3:hr:string, 4:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                    Select Vectorization:
-                        className: VectorSelectOperator
+                    Filter Vectorization:
+                        className: VectorFilterOperator
                         native: true
-                        projectedOutputColumnNums: [0, 1, 2, 3]
-                      Group By Vectorization:
-                          className: VectorGroupByOperator
-                          groupByMode: HASH
-                          keyExpressions: col 2:string, col 3:string, col 0:string, col 1:string
-                          native: false
-                          vectorProcessingMode: HASH
-                          projectedOutputColumnNums: []
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 0:string), SelectColumnIsNotNull(col 1:string))
+                      Select Vectorization:
+                          className: VectorSelectOperator
+                          native: true
+                          projectedOutputColumnNums: [0, 1, 2, 3, 4]
                         Reduce Sink Vectorization:
                             className: VectorReduceSinkMultiKeyOperator
                             keyColumns: 0:string, 1:string, 2:string, 3:string
                             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
+                            valueColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
             Map Vectorization:
@@ -1805,27 +1803,38 @@ STAGE PLANS:
                 inputFormatFeatureSupport: [DECIMAL_64]
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: false
+                allNative: true
                 usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
                     includeColumns: [0, 1]
                     dataColumns: key:string, value:string
+                    neededVirtualColumns: [ROWID]
                     partitionColumnCount: 2
                     partitionColumns: ds:string, hr:string
                     scratchColumnTypeNames: []
-        Map 8 
+        Map 7 
             Map Operator Tree:
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:key:string, 1:value:string, 2:ds:string, 3:hr:string, 4:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                    Reduce Sink Vectorization:
-                        className: VectorReduceSinkMultiKeyOperator
-                        keyColumns: 2:string, 3:string, 0:string, 1:string
+                    Select Vectorization:
+                        className: VectorSelectOperator
                         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
-                        valueColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
+                        projectedOutputColumnNums: [0, 1, 2, 3]
+                      Group By Vectorization:
+                          className: VectorGroupByOperator
+                          groupByMode: HASH
+                          keyExpressions: col 0:string, col 1:string, col 2:string, col 3:string
+                          native: false
+                          vectorProcessingMode: HASH
+                          projectedOutputColumnNums: []
+                        Reduce Sink Vectorization:
+                            className: VectorReduceSinkMultiKeyOperator
+                            keyColumns: 0:string, 1:string, 2:string, 3:string
+                            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
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
             Map Vectorization:
@@ -1834,78 +1843,21 @@ STAGE PLANS:
                 inputFormatFeatureSupport: [DECIMAL_64]
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: true
+                allNative: false
                 usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
                     includeColumns: [0, 1]
                     dataColumns: key:string, value:string
-                    neededVirtualColumns: [ROWID]
                     partitionColumnCount: 2
                     partitionColumns: ds:string, hr:string
                     scratchColumnTypeNames: []
         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
-                reduceColumnNullOrder: zzzz
-                reduceColumnSortOrder: ++++
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 4
-                    dataColumns: KEY._col0:string, KEY._col1:string, KEY._col2:string, KEY._col3:string
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-                Group By Vectorization:
-                    className: VectorGroupByOperator
-                    groupByMode: MERGEPARTIAL
-                    keyExpressions: col 0:string, col 1:string, col 2:string, col 3:string
-                    native: false
-                    vectorProcessingMode: MERGE_PARTIAL
-                    projectedOutputColumnNums: []
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkMultiKeyOperator
-                      keyColumns: 0:string, 1:string, 2:string, 3:string
-                      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
-                  Select Vectorization:
-                      className: VectorSelectOperator
-                      native: true
-                      projectedOutputColumnNums: [0]
-                    Group By Vectorization:
-                        className: VectorGroupByOperator
-                        groupByMode: HASH
-                        keyExpressions: col 0:string
-                        native: false
-                        vectorProcessingMode: HASH
-                        projectedOutputColumnNums: []
-                      App Master Event Vectorization:
-                          className: VectorAppMasterEventOperator
-                          native: true
-                  Select Vectorization:
-                      className: VectorSelectOperator
-                      native: true
-                      projectedOutputColumnNums: [1]
-                    Group By Vectorization:
-                        className: VectorGroupByOperator
-                        groupByMode: HASH
-                        keyExpressions: col 1:string
-                        native: false
-                        vectorProcessingMode: HASH
-                        projectedOutputColumnNums: []
-                      App Master Event Vectorization:
-                          className: VectorAppMasterEventOperator
-                          native: true
-        Reducer 3 
             MergeJoin Vectorization:
                 enabled: false
                 enableConditionsNotMet: Vectorizing MergeJoin Supported IS false
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
@@ -1928,6 +1880,14 @@ STAGE PLANS:
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
+        Reducer 4 
+            Execution mode: llap
+            Reduce Vectorization:
+                enabled: true
+                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
+                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_stats not supported
+                vectorized: false
+            Reduce Operator Tree:
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Vectorization:
@@ -1959,14 +1919,66 @@ STAGE PLANS:
                 notVectorizedReason: Key expression for GROUPBY operator: Vectorizing complex type STRUCT not supported
                 vectorized: false
             Reduce Operator Tree:
-        Reducer 7 
-            Execution mode: llap
+        Reducer 8 
+            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
-                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_stats not supported
-                vectorized: false
+                reduceColumnNullOrder: zzzz
+                reduceColumnSortOrder: ++++
+                allNative: false
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 4
+                    dataColumns: KEY._col0:string, KEY._col1:string, KEY._col2:string, KEY._col3:string
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
             Reduce Operator Tree:
+                Group By Vectorization:
+                    className: VectorGroupByOperator
+                    groupByMode: MERGEPARTIAL
+                    keyExpressions: col 0:string, col 1:string, col 2:string, col 3:string
+                    native: false
+                    vectorProcessingMode: MERGE_PARTIAL
+                    projectedOutputColumnNums: []
+                  Select Vectorization:
+                      className: VectorSelectOperator
+                      native: true
+                      projectedOutputColumnNums: [2, 3, 0, 1]
+                    Reduce Sink Vectorization:
+                        className: VectorReduceSinkMultiKeyOperator
+                        keyColumns: 0:string, 1:string, 2:string, 3:string
+                        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
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumnNums: [2]
+                      Group By Vectorization:
+                          className: VectorGroupByOperator
+                          groupByMode: HASH
+                          keyExpressions: col 2:string
+                          native: false
+                          vectorProcessingMode: HASH
+                          projectedOutputColumnNums: []
+                        App Master Event Vectorization:
+                            className: VectorAppMasterEventOperator
+                            native: true
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumnNums: [3]
+                      Group By Vectorization:
+                          className: VectorGroupByOperator
+                          groupByMode: HASH
+                          keyExpressions: col 3:string
+                          native: false
+                          vectorProcessingMode: HASH
+                          projectedOutputColumnNums: []
+                        App Master Event Vectorization:
+                            className: VectorAppMasterEventOperator
+                            native: true
 
   Stage: Stage-5
 
@@ -1974,15 +1986,15 @@ STAGE PLANS:
 
   Stage: Stage-6
 
-  Stage: Stage-2
+  Stage: Stage-1
 
   Stage: Stage-7
 
-  Stage: Stage-3
+  Stage: Stage-2
 
   Stage: Stage-8
 
-  Stage: Stage-1
+  Stage: Stage-3
 
   Stage: Stage-9
 
@@ -2230,19 +2242,19 @@ STAGE PLANS:
                     Filter Vectorization:
                         className: VectorFilterOperator
                         native: true
-                        predicateExpression: FilterExprOrExpr(children: FilterLongColEqualLongScalar(col 5:int, val 413)(children: CastStringToLong(col 0:string) -> 5:int), FilterLongColEqualLongScalar(col 6:int, val 43)(children: CastStringToLong(col 0:string) -> 6:int))
+                        predicateExpression: FilterLongColumnInList(col 5:int, values [413, 43])(children: CastStringToLong(col 0:string) -> 5:int)
                       Select Vectorization:
                           className: VectorSelectOperator
                           native: true
-                          projectedOutputColumnNums: [4, 0, 7, 2]
-                          selectExpressions: StringGroupColConcatStringScalar(col 1:string, val updated) -> 7:string
+                          projectedOutputColumnNums: [4, 0, 6, 2]
+                          selectExpressions: StringGroupColConcatStringScalar(col 1:string, val updated) -> 6:string
                         Reduce Sink Vectorization:
                             className: VectorReduceSinkObjectHashOperator
                             keyColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
                             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
                             partitionColumns: 5:int
-                            valueColumns: 0:string, 7:string, 2:string
+                            valueColumns: 0:string, 6:string, 2:string
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
             Map Vectorization:
@@ -2261,7 +2273,7 @@ STAGE PLANS:
                     neededVirtualColumns: [ROWID]
                     partitionColumnCount: 2
                     partitionColumns: ds:string, hr:string
-                    scratchColumnTypeNames: [bigint, bigint, string]
+                    scratchColumnTypeNames: [bigint, string]
         Reducer 2 
             Execution mode: vectorized, llap
             Reduce Vectorization:
@@ -2617,46 +2629,44 @@ STAGE DEPENDENCIES:
   Stage-5 depends on stages: Stage-4
   Stage-0 depends on stages: Stage-5
   Stage-6 depends on stages: Stage-0
+  Stage-1 depends on stages: Stage-5
+  Stage-7 depends on stages: Stage-1
   Stage-2 depends on stages: Stage-5
-  Stage-7 depends on stages: Stage-2
+  Stage-8 depends on stages: Stage-2
   Stage-3 depends on stages: Stage-5
-  Stage-8 depends on stages: Stage-3
-  Stage-1 depends on stages: Stage-5
-  Stage-9 depends on stages: Stage-1
+  Stage-9 depends on stages: Stage-3
 
 STAGE PLANS:
   Stage: Stage-4
     Tez
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
-        Reducer 3 <- Map 9 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 6 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 3 (SIMPLE_EDGE)
-        Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 7 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 9 <- Map 8 (SIMPLE_EDGE)
       Vertices:
         Map 1 
             Map Operator Tree:
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:key:string, 1:value:string, 2:ds:string, 3:hr:string, 4:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                    Select Vectorization:
-                        className: VectorSelectOperator
+                    Filter Vectorization:
+                        className: VectorFilterOperator
                         native: true
-                        projectedOutputColumnNums: [0, 1, 2, 3]
-                      Group By Vectorization:
-                          className: VectorGroupByOperator
-                          groupByMode: HASH
-                          keyExpressions: col 2:string, col 3:string, col 0:string, col 1:string
-                          native: false
-                          vectorProcessingMode: HASH
-                          projectedOutputColumnNums: []
+                        predicateExpression: FilterExprAndExpr(children: SelectColumnIsNotNull(col 0:string), SelectColumnIsNotNull(col 1:string))
+                      Select Vectorization:
+                          className: VectorSelectOperator
+                          native: true
+                          projectedOutputColumnNums: [0, 1, 2, 3, 4]
                         Reduce Sink Vectorization:
                             className: VectorReduceSinkMultiKeyOperator
                             keyColumns: 0:string, 1:string, 2:string, 3:string
                             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
+                            valueColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
             Map Vectorization:
@@ -2665,27 +2675,38 @@ STAGE PLANS:
                 inputFormatFeatureSupport: [DECIMAL_64]
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: false
+                allNative: true
                 usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
                     includeColumns: [0, 1]
                     dataColumns: key:string, value:string
+                    neededVirtualColumns: [ROWID]
                     partitionColumnCount: 2
                     partitionColumns: ds:string, hr:string
                     scratchColumnTypeNames: []
-        Map 9 
+        Map 8 
             Map Operator Tree:
                   TableScan Vectorization:
                       native: true
                       vectorizationSchemaColumns: [0:key:string, 1:value:string, 2:ds:string, 3:hr:string, 4:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
-                    Reduce Sink Vectorization:
-                        className: VectorReduceSinkMultiKeyOperator
-                        keyColumns: 2:string, 3:string, 0:string, 1:string
+                    Select Vectorization:
+                        className: VectorSelectOperator
                         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
-                        valueColumns: 4:struct<writeid:bigint,bucketid:int,rowid:bigint>
+                        projectedOutputColumnNums: [0, 1, 2, 3]
+                      Group By Vectorization:
+                          className: VectorGroupByOperator
+                          groupByMode: HASH
+                          keyExpressions: col 0:string, col 1:string, col 2:string, col 3:string
+                          native: false
+                          vectorProcessingMode: HASH
+                          projectedOutputColumnNums: []
+                        Reduce Sink Vectorization:
+                            className: VectorReduceSinkMultiKeyOperator
+                            keyColumns: 0:string, 1:string, 2:string, 3:string
+                            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
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
             Map Vectorization:
@@ -2694,78 +2715,21 @@ STAGE PLANS:
                 inputFormatFeatureSupport: [DECIMAL_64]
                 featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                allNative: true
+                allNative: false
                 usesVectorUDFAdaptor: false
                 vectorized: true
                 rowBatchContext:
                     dataColumnCount: 2
                     includeColumns: [0, 1]
                     dataColumns: key:string, value:string
-                    neededVirtualColumns: [ROWID]
                     partitionColumnCount: 2
                     partitionColumns: ds:string, hr:string
                     scratchColumnTypeNames: []
         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
-                reduceColumnNullOrder: zzzz
-                reduceColumnSortOrder: ++++
-                allNative: false
-                usesVectorUDFAdaptor: false
-                vectorized: true
-                rowBatchContext:
-                    dataColumnCount: 4
-                    dataColumns: KEY._col0:string, KEY._col1:string, KEY._col2:string, KEY._col3:string
-                    partitionColumnCount: 0
-                    scratchColumnTypeNames: []
-            Reduce Operator Tree:
-                Group By Vectorization:
-                    className: VectorGroupByOperator
-                    groupByMode: MERGEPARTIAL
-                    keyExpressions: col 0:string, col 1:string, col 2:string, col 3:string
-                    native: false
-                    vectorProcessingMode: MERGE_PARTIAL
-                    projectedOutputColumnNums: []
-                  Reduce Sink Vectorization:
-                      className: VectorReduceSinkMultiKeyOperator
-                      keyColumns: 0:string, 1:string, 2:string, 3:string
-                      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
-                  Select Vectorization:
-                      className: VectorSelectOperator
-                      native: true
-                      projectedOutputColumnNums: [0]
-                    Group By Vectorization:
-                        className: VectorGroupByOperator
-                        groupByMode: HASH
-                        keyExpressions: col 0:string
-                        native: false
-                        vectorProcessingMode: HASH
-                        projectedOutputColumnNums: []
-                      App Master Event Vectorization:
-                          className: VectorAppMasterEventOperator
-                          native: true
-                  Select Vectorization:
-                      className: VectorSelectOperator
-                      native: true
-                      projectedOutputColumnNums: [1]
-                    Group By Vectorization:
-                        className: VectorGroupByOperator
-                        groupByMode: HASH
-                        keyExpressions: col 1:string
-                        native: false
-                        vectorProcessingMode: HASH
-                        projectedOutputColumnNums: []
-                      App Master Event Vectorization:
-                          className: VectorAppMasterEventOperator
-                          native: true
-        Reducer 3 
             MergeJoin Vectorization:
                 enabled: false
                 enableConditionsNotMet: Vectorizing MergeJoin Supported IS false
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
@@ -2788,7 +2752,23 @@ STAGE PLANS:
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
+        Reducer 4 
+            Execution mode: llap
+            Reduce Vectorization:
+                enabled: true
+                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
+                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_stats not supported
+                vectorized: false
+            Reduce Operator Tree:
         Reducer 5 
+            Execution mode: llap
+            Reduce Vectorization:
+                enabled: true
+                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
+                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_stats not supported
+                vectorized: false
+            Reduce Operator Tree:
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
@@ -2811,30 +2791,74 @@ STAGE PLANS:
                   File Sink Vectorization:
                       className: VectorFileSinkOperator
                       native: false
-        Reducer 6 
-            Execution mode: llap
-            Reduce Vectorization:
-                enabled: true
-                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                notVectorizedReason: Key expression for GROUPBY operator: Vectorizing complex type STRUCT not supported
-                vectorized: false
-            Reduce Operator Tree:
         Reducer 7 
             Execution mode: llap
             Reduce Vectorization:
                 enabled: true
                 enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_stats not supported
+                notVectorizedReason: Key expression for GROUPBY operator: Vectorizing complex type STRUCT not supported
                 vectorized: false
             Reduce Operator Tree:
-        Reducer 8 
-            Execution mode: llap
+        Reducer 9 
+            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
-                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_stats not supported
-                vectorized: false
+                reduceColumnNullOrder: zzzz
+                reduceColumnSortOrder: ++++
+                allNative: false
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 4
+                    dataColumns: KEY._col0:string, KEY._col1:string, KEY._col2:string, KEY._col3:string
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
             Reduce Operator Tree:
+                Group By Vectorization:
+                    className: VectorGroupByOperator
+                    groupByMode: MERGEPARTIAL
+                    keyExpressions: col 0:string, col 1:string, col 2:string, col 3:string
+                    native: false
+                    vectorProcessingMode: MERGE_PARTIAL
+                    projectedOutputColumnNums: []
+                  Select Vectorization:
+                      className: VectorSelectOperator
+                      native: true
+                      projectedOutputColumnNums: [2, 3, 0, 1]
+                    Reduce Sink Vectorization:
+                        className: VectorReduceSinkMultiKeyOperator
+                        keyColumns: 0:string, 1:string, 2:string, 3:string
+                        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
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumnNums: [2]
+                      Group By Vectorization:
+                          className: VectorGroupByOperator
+                          groupByMode: HASH
+                          keyExpressions: col 2:string
+                          native: false
+                          vectorProcessingMode: HASH
+                          projectedOutputColumnNums: []
+                        App Master Event Vectorization:
+                            className: VectorAppMasterEventOperator
+                            native: true
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumnNums: [3]
+                      Group By Vectorization:
+                          className: VectorGroupByOperator
+                          groupByMode: HASH
+                          keyExpressions: col 3:string
+                          native: false
+                          vectorProcessingMode: HASH
+                          projectedOutputColumnNums: []
+                        App Master Event Vectorization:
+                            className: VectorAppMasterEventOperator
+                            native: true
 
   Stage: Stage-5
 
@@ -2842,15 +2866,15 @@ STAGE PLANS:
 
   Stage: Stage-6
 
-  Stage: Stage-2
+  Stage: Stage-1
 
   Stage: Stage-7
 
-  Stage: Stage-3
+  Stage: Stage-2
 
   Stage: Stage-8
 
-  Stage: Stage-1
+  Stage: Stage-3
 
   Stage: Stage-9
 
diff --git a/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out b/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
index 2e43110..ef837da 100644
--- a/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_vectorization_original.q.out
@@ -585,21 +585,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: over10k_orc_bucketed
-                  filterExpr: ((b = 4294967363L) and (t < 100Y)) (type: boolean)
+                  filterExpr: ((t < 100Y) and (b = 4294967363L)) (type: boolean)
                   Statistics: Num rows: 2098 Data size: 706986 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: ((b = 4294967363L) and (t < 100Y)) (type: boolean)
-                    Statistics: Num rows: 6 Data size: 2022 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: ((t < 100Y) and (b = 4294967363L)) (type: boolean)
+                    Statistics: Num rows: 7 Data size: 2359 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: ROW__ID (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), t (type: tinyint), si (type: smallint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary)
                       outputColumnNames: _col0, _col1, _col2, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                      Statistics: Num rows: 6 Data size: 2502 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 2919 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>)
                         null sort order: z
                         sort order: +
                         Map-reduce partition columns: UDFToInteger(_col0) (type: int)
-                        Statistics: Num rows: 6 Data size: 2502 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 2919 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: tinyint), _col2 (type: smallint), _col5 (type: float), _col6 (type: double), _col7 (type: boolean), _col8 (type: string), _col9 (type: timestamp), _col10 (type: decimal(4,2)), _col11 (type: binary)
             Execution mode: vectorized, llap
             LLAP IO: may be used (ACID table)
@@ -607,12 +607,12 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
-                expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), 0 (type: int), 4294967363L (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: boolean), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: decimal(4,2)), VALUE._col9 (type: binary)
+                expressions: KEY.reducesinkkey0 (type: struct<writeid:bigint,bucketid:int,rowid:bigint>), VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), 0 (type: int), 4294967363L (type: bigint), VALUE._col2 (type: float), VALUE._col3 (type: double), VALUE._col4 (type: boolean), VALUE._col5 (type: string), VALUE._col6 (type: timestamp), VALUE._col7 (type: decimal(4,2)), VALUE._col8 (type: binary)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-                Statistics: Num rows: 6 Data size: 2502 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 2919 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 6 Data size: 2502 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7 Data size: 2919 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                       output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/auto_join0.q.out b/ql/src/test/results/clientpositive/llap/auto_join0.q.out
index 31b776f..4fbd217 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join0.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join0.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[22][tables = [src1, src2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain 
 select sum(hash(a.k1,a.v1,a.k2, a.v2))
 from (
@@ -44,10 +44,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
@@ -88,17 +88,21 @@ STAGE PLANS:
                   1 
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  aggregations: sum(hash(_col0,_col1,_col2,_col3))
-                  minReductionHashAggr: 0.99
-                  mode: hash
+                Select Operator
+                  expressions: hash(_col0,_col1,_col2,_col3) (type: int)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    null sort order: 
-                    sort order: 
+                  Statistics: Num rows: 27556 Data size: 9809936 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: sum(_col0)
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: bigint)
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: bigint)
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -133,7 +137,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[22][tables = [src1, src2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select sum(hash(a.k1,a.v1,a.k2, a.v2))
 from (
 SELECT src1.key as k1, src1.value as v1, 
diff --git a/ql/src/test/results/clientpositive/llap/auto_join21.q.out b/ql/src/test/results/clientpositive/llap/auto_join21.q.out
index df866d8..21e5e5e 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join21.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join21.q.out
@@ -27,44 +27,51 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Left Outer Join 0 to 1
-                    filter predicates:
-                      0 {(key < 10)}
-                      1 
-                    keys:
-                      0 key (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6
-                    input vertices:
-                      1 Map 2
-                    Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col5 (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col5 (type: string)
-                      Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1, _col2, _col3
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col2 (type: string)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col2 (type: string)
+                          Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: (key > 10) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (key > 10) (type: boolean)
-                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
-                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -72,28 +79,28 @@ STAGE PLANS:
                 TableScan
                   alias: src3
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Right Outer Join 0 to 1
-                    filter predicates:
-                      0 
-                      1 {(key < 10)}
-                    keys:
-                      0 _col5 (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                    input vertices:
-                      0 Map 1
-                    Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Right Outer Join 0 to 1
+                      filter predicates:
+                        0 
+                        1 {_col2}
+                      keys:
+                        0 _col2 (type: string)
+                        1 _col0 (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                      Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                      input vertices:
+                        0 Map 1
+                      Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                         null sort order: zzzzzz
                         sort order: ++++++
-                        Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 4 
@@ -102,10 +109,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/auto_join29.q.out b/ql/src/test/results/clientpositive/llap/auto_join29.q.out
index 9c3a0b1..dc560dc 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join29.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join29.q.out
@@ -27,44 +27,51 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Left Outer Join 0 to 1
-                    filter predicates:
-                      0 {(key < 10)}
-                      1 
-                    keys:
-                      0 key (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6
-                    input vertices:
-                      1 Map 2
-                    Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col5 (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col5 (type: string)
-                      Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1, _col2, _col3
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col2 (type: string)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col2 (type: string)
+                          Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: (key > 10) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (key > 10) (type: boolean)
-                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
-                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -72,28 +79,28 @@ STAGE PLANS:
                 TableScan
                   alias: src3
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Right Outer Join 0 to 1
-                    filter predicates:
-                      0 
-                      1 {(key < 10)}
-                    keys:
-                      0 _col5 (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                    input vertices:
-                      0 Map 1
-                    Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Right Outer Join 0 to 1
+                      filter predicates:
+                        0 
+                        1 {_col2}
+                      keys:
+                        0 _col2 (type: string)
+                        1 _col0 (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                      Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                      input vertices:
+                        0 Map 1
+                      Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                         null sort order: zzzzzz
                         sort order: ++++++
-                        Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 4 
@@ -102,10 +109,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -653,74 +660,82 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Left Outer Join 0 to 1
-                    filter predicates:
-                      0 {(key < 10)}
-                      1 
-                    keys:
-                      0 key (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6
-                    input vertices:
-                      1 Map 3
-                    Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Map Join Operator
                       condition map:
                            Left Outer Join 0 to 1
+                      filter predicates:
+                        0 {(UDFToDouble(_col0) < 10.0D)}
+                        1 
                       keys:
-                        0 _col5 (type: string)
-                        1 key (type: string)
-                      outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
+                        0 _col0 (type: string)
+                        1 _col0 (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3
                       input vertices:
-                        1 Map 4
-                      Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
-                      Select Operator
-                        expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                        1 Map 3
+                      Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Left Outer Join 0 to 1
+                        keys:
+                          0 _col2 (type: string)
+                          1 _col0 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                        input vertices:
+                          1 Map 4
+                        Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                           null sort order: zzzzzz
                           sort order: ++++++
-                          Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: (key > 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) > 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key > 10) (type: boolean)
+                    predicate: (UDFToDouble(key) > 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: src3
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -729,10 +744,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1279,18 +1294,22 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src1
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
@@ -1298,56 +1317,60 @@ STAGE PLANS:
                 TableScan
                   alias: src2
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Right Outer Join 0 to 1
-                    filter predicates:
-                      0 
-                      1 {(key > 10)}
-                    keys:
-                      0 key (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6
-                    input vertices:
-                      0 Map 1
-                    Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Map Join Operator
                       condition map:
-                           Left Outer Join 0 to 1
+                           Right Outer Join 0 to 1
+                      filter predicates:
+                        0 
+                        1 {(UDFToDouble(_col0) > 10.0D)}
                       keys:
-                        0 _col5 (type: string)
-                        1 key (type: string)
-                      outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
+                        0 _col0 (type: string)
+                        1 _col0 (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3
                       input vertices:
-                        1 Map 4
-                      Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
-                      Select Operator
-                        expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                        0 Map 1
+                      Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Left Outer Join 0 to 1
+                        keys:
+                          0 _col2 (type: string)
+                          1 _col0 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                        input vertices:
+                          1 Map 4
+                        Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                           null sort order: zzzzzz
                           sort order: ++++++
-                          Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: src3
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -1356,10 +1379,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1204 Data size: 554114 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 762 Data size: 318086 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1910,54 +1933,70 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 2 <- Map 1 (BROADCAST_EDGE)
-        Map 3 <- Map 2 (BROADCAST_EDGE)
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+        Map 3 <- Map 1 (BROADCAST_EDGE)
         Reducer 4 <- Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: src1
-                  filterExpr: (key < 10) (type: boolean)
+                  alias: src2
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
-                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
-                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string), (UDFToDouble(key) > 10.0D) (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Left Outer Join 0 to 1
+                        filter predicates:
+                          0 {_col2}
+                          1 
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1, _col3, _col4
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                        Select Operator
+                          expressions: _col3 (type: string), _col4 (type: string), _col0 (type: string), _col1 (type: string)
+                          outputColumnNames: _col0, _col1, _col2, _col3
+                          Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            key expressions: _col2 (type: string)
+                            null sort order: z
+                            sort order: +
+                            Map-reduce partition columns: _col2 (type: string)
+                            Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
             Map Operator Tree:
                 TableScan
-                  alias: src2
+                  alias: src1
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Right Outer Join 0 to 1
-                    filter predicates:
-                      0 
-                      1 {(key > 10)}
-                    keys:
-                      0 key (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6
-                    input vertices:
-                      0 Map 1
-                    Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col5 (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col5 (type: string)
-                      Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+                  Filter Operator
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -1965,28 +2004,28 @@ STAGE PLANS:
                 TableScan
                   alias: src3
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Right Outer Join 0 to 1
-                    filter predicates:
-                      0 
-                      1 {(key < 10)}
-                    keys:
-                      0 _col5 (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                    input vertices:
-                      0 Map 2
-                    Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Right Outer Join 0 to 1
+                      filter predicates:
+                        0 
+                        1 {_col2}
+                      keys:
+                        0 _col2 (type: string)
+                        1 _col0 (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                      Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+                      input vertices:
+                        0 Map 1
+                      Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                         null sort order: zzzzzz
                         sort order: ++++++
-                        Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 4 
@@ -1995,10 +2034,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1705 Data size: 643826 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1705 Data size: 687326 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2557,76 +2596,82 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src1
-                  filterExpr: ((key > 10) and (key < 10)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key > 10) and (key < 10)) (type: boolean)
-                    Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                    Map Join Operator
-                      condition map:
-                           Inner Join 0 to 1
-                      keys:
-                        0 key (type: string)
-                        1 key (type: string)
-                      outputColumnNames: _col0, _col1, _col5, _col6
-                      input vertices:
-                        1 Map 3
-                      Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
-                             Left Outer Join 0 to 1
+                             Inner Join 0 to 1
                         keys:
-                          0 _col5 (type: string)
-                          1 key (type: string)
-                        outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                        outputColumnNames: _col0, _col1, _col2, _col3
                         input vertices:
-                          1 Map 4
-                        Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
-                        Select Operator
-                          expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                          1 Map 3
+                        Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map Join Operator
+                          condition map:
+                               Left Outer Join 0 to 1
+                          keys:
+                            0 _col2 (type: string)
+                            1 _col0 (type: string)
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                          Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
+                          input vertices:
+                            1 Map 4
+                          Statistics: Num rows: 2 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                             null sort order: zzzzzz
                             sort order: ++++++
-                            Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 2 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: ((key < 10) and (key > 10)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key < 10) and (key > 10)) (type: boolean)
-                    Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
-                      Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: src3
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -2635,10 +2680,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 221 Data size: 102884 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 1068 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2677,8 +2722,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 2 (BROADCAST_EDGE)
-        Map 3 <- Map 1 (BROADCAST_EDGE)
+        Map 2 <- Map 1 (BROADCAST_EDGE)
+        Map 3 <- Map 2 (BROADCAST_EDGE)
         Reducer 4 <- Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
@@ -2686,46 +2731,55 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src1
-                  filterExpr: ((key > 10) and (key < 10)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key > 10) and (key < 10)) (type: boolean)
-                    Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                    Map Join Operator
-                      condition map:
-                           Inner Join 0 to 1
-                      keys:
-                        0 key (type: string)
-                        1 key (type: string)
-                      outputColumnNames: _col0, _col1, _col5, _col6
-                      input vertices:
-                        1 Map 2
-                      Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col5 (type: string)
+                        key expressions: _col0 (type: string)
                         null sort order: z
                         sort order: +
-                        Map-reduce partition columns: _col5 (type: string)
-                        Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: ((key < 10) and (key > 10)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key < 10) and (key > 10)) (type: boolean)
-                    Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
-                      Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: _col0 is not null (type: boolean)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col0 (type: string)
+                            1 _col0 (type: string)
+                          outputColumnNames: _col0, _col1, _col2, _col3
+                          input vertices:
+                            0 Map 1
+                          Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            key expressions: _col2 (type: string)
+                            null sort order: z
+                            sort order: +
+                            Map-reduce partition columns: _col2 (type: string)
+                            Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -2733,28 +2787,28 @@ STAGE PLANS:
                 TableScan
                   alias: src3
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Right Outer Join 0 to 1
-                    filter predicates:
-                      0 
-                      1 {(key < 10)}
-                    keys:
-                      0 _col5 (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                    input vertices:
-                      0 Map 1
-                    Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string), (UDFToDouble(key) < 10.0D) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 500 Data size: 91000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Right Outer Join 0 to 1
+                      filter predicates:
+                        0 
+                        1 {_col2}
+                      keys:
+                        0 _col2 (type: string)
+                        1 _col0 (type: string)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                      Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+                      input vertices:
+                        0 Map 2
+                      Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                         null sort order: zzzzzz
                         sort order: ++++++
-                        Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 4 
@@ -2763,10 +2817,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 636 Data size: 161980 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 501 Data size: 89890 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3305,8 +3359,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
-        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+        Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
+        Reducer 3 <- Map 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -3314,89 +3368,96 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Map Join Operator
-                    condition map:
-                         Left Outer Join 0 to 1
-                    filter predicates:
-                      0 {(key < 10)}
-                      1 
-                    keys:
-                      0 key (type: string)
-                      1 key (type: string)
-                    outputColumnNames: _col0, _col1, _col5, _col6
-                    input vertices:
-                      1 Map 3
-                    Statistics: Num rows: 762 Data size: 182450 Basic stats: COMPLETE Column stats: COMPLETE
-                    Filter Operator
-                      predicate: (_col5 < 10) (type: boolean)
-                      Statistics: Num rows: 254 Data size: 60876 Basic stats: COMPLETE Column stats: COMPLETE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col5 (type: string)
-                          1 key (type: string)
-                        outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                        input vertices:
-                          1 Map 4
-                        Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
-                        Select Operator
-                          expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-                          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                          Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
-                          Reduce Output Operator
-                            key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
-                            null sort order: zzzzzz
-                            sort order: ++++++
-                            Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 2 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: (key > 10) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (key > 10) (type: boolean)
-                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
-                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: (UDFToDouble(_col0) < 10.0D) (type: boolean)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col0 (type: string)
+                            1 _col0 (type: string)
+                          outputColumnNames: _col0, _col1, _col2, _col3
+                          input vertices:
+                            0 Map 1
+                          Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Map Join Operator
+                            condition map:
+                                 Inner Join 0 to 1
+                            keys:
+                              0 _col2 (type: string)
+                              1 _col0 (type: string)
+                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                            input vertices:
+                              1 Map 4
+                            Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
+                            Reduce Output Operator
+                              key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                              null sort order: zzzzzz
+                              sort order: ++++++
+                              Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: src3
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Reducer 2 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 401 Data size: 184586 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3416,6 +3477,7 @@ POSTHOOK: query: SELECT * FROM src src1 LEFT OUTER JOIN src src2 ON (src1.key =
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 #### A masked pattern was here ####
+Warning: Map Join MAPJOIN[35][bigTable=?] in task 'Map 2' is a cross product
 PREHOOK: query: explain
 SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value
 PREHOOK: type: QUERY
@@ -3443,79 +3505,84 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src1
-                  filterExpr: (key < 10) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (key < 10) (type: boolean)
-                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
-                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Map Join Operator
-                      condition map:
-                           Right Outer Join 0 to 1
-                      filter predicates:
-                        0 
-                        1 {(key > 10)}
-                      keys:
-                        0 key (type: string)
-                        1 key (type: string)
-                      outputColumnNames: _col0, _col1, _col5, _col6
-                      input vertices:
-                        0 Map 1
-                      Statistics: Num rows: 262 Data size: 93272 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                       Map Join Operator
                         condition map:
-                             Inner Join 0 to 1
+                             Right Outer Join 0 to 1
+                        filter predicates:
+                          0 
+                          1 {true}
                         keys:
-                          0 _col5 (type: string)
-                          1 key (type: string)
-                        outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
+                          0 
+                          1 
+                        outputColumnNames: _col0, _col1, _col2, _col3
                         input vertices:
-                          1 Map 4
-                        Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
-                        Select Operator
-                          expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
+                          0 Map 1
+                        Statistics: Num rows: 166 Data size: 59096 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col2 (type: string)
+                            1 _col0 (type: string)
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                          Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
+                          input vertices:
+                            1 Map 4
+                          Statistics: Num rows: 166 Data size: 88644 Basic stats: COMPLETE Column stats: COMPLETE
                           Reduce Output Operator
                             key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
                             null sort order: zzzzzz
                             sort order: ++++++
-                            Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
+                            Statistics: Num rows: 166 Data size: 88644 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: src3
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -3524,10 +3591,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 166 Data size: 88644 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 414 Data size: 221076 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 166 Data size: 88644 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -3539,6 +3606,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
+Warning: Map Join MAPJOIN[35][bigTable=?] in task 'Map 2' is a cross product
 PREHOOK: query: SELECT * FROM src src1 RIGHT OUTER JOIN src src2 ON (src1.key = src2.key AND src1.key < 10 AND src2.key > 10) JOIN src src3 ON (src2.key = src3.key AND src3.key < 10) SORT BY src1.key, src1.value, src2.key, src2.value, src3.key, src3.value
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -3588,97 +3656,105 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 2 (BROADCAST_EDGE)
-        Map 3 <- Map 1 (BROADCAST_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+        Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
+        Reducer 3 <- Map 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: src1
-                  filterExpr: ((key > 10) and (key < 10)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key > 10) and (key < 10)) (type: boolean)
-                    Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                    Map Join Operator
-                      condition map:
-                           Inner Join 0 to 1
-                      keys:
-                        0 key (type: string)
-                        1 key (type: string)
-                      outputColumnNames: _col0, _col1, _col5, _col6
-                      input vertices:
-                        1 Map 2
-                      Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col5 (type: string)
+                        key expressions: _col0 (type: string)
                         null sort order: z
                         sort order: +
-                        Map-reduce partition columns: _col5 (type: string)
-                        Statistics: Num rows: 86 Data size: 30616 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string), _col1 (type: string), _col6 (type: string)
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
             Map Operator Tree:
                 TableScan
                   alias: src2
-                  filterExpr: ((key < 10) and (key > 10)) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: ((key < 10) and (key > 10)) (type: boolean)
-                    Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: key (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: key (type: string)
-                      Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: value (type: string)
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 0
+                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: (UDFToDouble(_col0) < 10.0D) (type: boolean)
+                        Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col0 (type: string)
+                            1 _col0 (type: string)
+                          outputColumnNames: _col0, _col1, _col2, _col3
+                          input vertices:
+                            0 Map 1
+                          Statistics: Num rows: 1 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                          Map Join Operator
+                            condition map:
+                                 Inner Join 0 to 1
+                            keys:
+                              0 _col2 (type: string)
+                              1 _col0 (type: string)
+                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                            input vertices:
+                              1 Map 4
+                            Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
+                            Reduce Output Operator
+                              key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+                              null sort order: zzzzzz
+                              sort order: ++++++
+                              Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Map 3 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: src3
-                  filterExpr: (key < 10) (type: boolean)
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key < 10) (type: boolean)
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Map Join Operator
-                      condition map:
-                           Inner Join 0 to 1
-                      keys:
-                        0 _col5 (type: string)
-                        1 key (type: string)
-                      outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11
-                      input vertices:
-                        0 Map 1
-                      Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
-                      Select Operator
-                        expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string)
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
-                          null sort order: zzzzzz
-                          sort order: ++++++
-                          Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Reducer 4 
+        Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 135 Data size: 72090 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 534 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/auto_join30.q.out b/ql/src/test/results/clientpositive/llap/auto_join30.q.out
index d81dadb..0a74992 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join30.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join30.q.out
@@ -71,21 +71,25 @@ STAGE PLANS:
                         keys:
                           0 _col0 (type: string)
                           1 _col0 (type: string)
-                        outputColumnNames: _col2, _col3
+                        outputColumnNames: _col1, _col2
                         input vertices:
                           0 Map 1
                         Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
-                        Group By Operator
-                          aggregations: sum(hash(_col2,_col3))
-                          minReductionHashAggr: 0.99
-                          mode: hash
+                        Select Operator
+                          expressions: hash(_col1,_col2) (type: int)
                           outputColumnNames: _col0
-                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                          Reduce Output Operator
-                            null sort order: 
-                            sort order: 
+                          Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
+                          Group By Operator
+                            aggregations: sum(_col0)
+                            minReductionHashAggr: 0.99
+                            mode: hash
+                            outputColumnNames: _col0
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                            value expressions: _col0 (type: bigint)
+                            Reduce Output Operator
+                              null sort order: 
+                              sort order: 
+                              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                              value expressions: _col0 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -177,39 +181,47 @@ STAGE PLANS:
                       keys:
                         0 _col0 (type: string)
                         1 _col0 (type: string)
-                      outputColumnNames: _col2, _col3
+                      outputColumnNames: _col1, _col2
                       input vertices:
                         1 Map 3
                       Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
-                      Group By Operator
-                        aggregations: sum(hash(_col2,_col3))
-                        minReductionHashAggr: 0.99
-                        mode: hash
+                      Select Operator
+                        expressions: hash(_col1,_col2) (type: int)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          null sort order: 
-                          sort order: 
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: sum(_col0)
+                          minReductionHashAggr: 0.99
+                          mode: hash
+                          outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                          value expressions: _col0 (type: bigint)
+                          Reduce Output Operator
+                            null sort order: 
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -290,17 +302,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: string)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
@@ -318,21 +334,25 @@ STAGE PLANS:
                       keys:
                         0 _col0 (type: string)
                         1 _col0 (type: string)
-                      outputColumnNames: _col2, _col3
+                      outputColumnNames: _col1, _col2
                       input vertices:
                         0 Map 1
                       Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
-                      Group By Operator
-                        aggregations: sum(hash(_col2,_col3))
-                        minReductionHashAggr: 0.99
-                        mode: hash
+                      Select Operator
+                        expressions: hash(_col1,_col2) (type: int)
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          null sort order: 
-                          sort order: 
+                        Statistics: Num rows: 791 Data size: 140798 Basic stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: sum(_col0)
+                          minReductionHashAggr: 0.99
+                          mode: hash
+                          outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                          value expressions: _col0 (type: bigint)
+                          Reduce Output Operator
+                            null sort order: 
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -411,8 +431,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 2 <- Map 1 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
-        Reducer 3 <- Map 2 (CUSTOM_SIMPLE_EDGE)
+        Map 1 <- Map 3 (BROADCAST_EDGE), Map 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -428,37 +448,16 @@ STAGE PLANS:
                       expressions: key (type: string)
                       outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
... 34176 lines suppressed ...