You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2019/04/01 08:03:25 UTC

[hive] branch master updated (dc0b16a -> f8a73a8)

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

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


    from dc0b16a  HIVE-21001: Upgrade to calcite-1.19 (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
     new 09c0de6  HIVE-21316: Comparision of varchar column and string literal should happen in varchar (Zoltan Haindrich reviewed by Vineet Garg)
     new f8a73a8  HIVE-21230: LEFT OUTER JOIN does not generate transitive IS NOT NULL filter on right side (HiveJoinAddNotNullRule bails out for outer joins) (Vineet Garg via Jesus Camacho Rodriguez)

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../test/resources/testconfiguration.properties    |   2 +
 .../calcite/rules/HiveJoinAddNotNullRule.java      |  83 ++-
 .../calcite/translator/ExprNodeConverter.java      |  24 +-
 .../calcite/translator/RexNodeConverter.java       |  26 +-
 .../hadoop/hive/ql/parse/TypeCheckProcFactory.java |  41 +-
 ql/src/test/queries/clientpositive/fold_varchar.q  |  17 +
 .../queries/clientpositive/transitive_not_null.q   |  53 ++
 .../clientpositive/annotate_stats_join.q.out       |  38 +-
 .../cbo_SortUnionTransposeRule.q.out               |  80 +--
 .../test/results/clientpositive/cbo_rp_join0.q.out |  30 +-
 .../results/clientpositive/constant_prop_3.q.out   |   4 +-
 .../clientpositive/correlationoptimizer8.q.out     |  40 +-
 .../clientpositive/in_typecheck_varchar.q.out      |   9 +-
 .../results/clientpositive/infer_join_preds.q.out  |  40 +-
 ql/src/test/results/clientpositive/innerjoin.q.out |  20 +-
 ql/src/test/results/clientpositive/join45.q.out    | 144 +++---
 ql/src/test/results/clientpositive/join46.q.out    | 200 +++----
 ql/src/test/results/clientpositive/join47.q.out    | 142 ++---
 .../join_cond_pushdown_unqual5.q.out               |  38 +-
 .../clientpositive/join_emit_interval.q.out        |  30 +-
 .../clientpositive/join_filters_overlap.q.out      |  66 +--
 .../test/results/clientpositive/join_merging.q.out |  38 +-
 ql/src/test/results/clientpositive/join_star.q.out | 162 +++---
 ql/src/test/results/clientpositive/lineage1.q.out  |  40 +-
 .../llap/auto_sortmerge_join_14.q.out              |  20 +-
 .../llap/auto_sortmerge_join_15.q.out              |  20 +-
 .../llap/auto_sortmerge_join_16.q.out              |  69 +--
 .../clientpositive/llap/check_constraint.q.out     |  22 +-
 .../clientpositive/llap/constprog_dpp.q.out        |  54 +-
 .../llap/correlationoptimizer1.q.out               | 180 ++++---
 .../llap/correlationoptimizer2.q.out               | 112 ++--
 .../llap/correlationoptimizer4.q.out               | 216 +++++---
 .../llap/dynamic_partition_pruning.q.out           |   4 +
 .../clientpositive/llap/explainuser_1.q.out        | 140 ++---
 .../results/clientpositive/llap/fold_varchar.q.out |  61 +++
 .../llap/hybridgrace_hashjoin_1.q.out              |  48 +-
 .../clientpositive/llap/join32_lessSize.q.out      |  28 +-
 .../test/results/clientpositive/llap/join46.q.out  | 212 ++++----
 .../clientpositive/llap/join_emit_interval.q.out   |  30 +-
 .../clientpositive/llap/limit_join_transpose.q.out | 344 +++++++-----
 .../results/clientpositive/llap/lineage2.q.out     |   2 +-
 .../results/clientpositive/llap/lineage3.q.out     |   4 +-
 .../results/clientpositive/llap/mapjoin3.q.out     |  22 +-
 .../results/clientpositive/llap/mapjoin46.q.out    | 322 ++++++------
 .../llap/mapjoin_emit_interval.q.out               |  30 +-
 .../llap/materialized_view_rewrite_1.q.out         |   8 +-
 .../llap/materialized_view_rewrite_6.q.out         |   4 +-
 .../materialized_view_rewrite_no_join_opt.q.out    |   4 +-
 .../results/clientpositive/llap/mergejoin.q.out    |  20 +-
 .../results/clientpositive/llap/sharedwork.q.out   |  58 ++-
 .../clientpositive/llap/skewjoinopt15.q.out        |  38 +-
 .../clientpositive/llap/smb_mapjoin_17.q.out       | 576 +++++++++++++--------
 .../clientpositive/llap/smb_mapjoin_4.q.out        | 264 +++++-----
 .../clientpositive/llap/smb_mapjoin_5.q.out        | 264 +++++-----
 .../clientpositive/llap/subquery_in_having.q.out   |  70 +--
 .../clientpositive/llap/subquery_multi.q.out       | 125 +++--
 .../clientpositive/llap/subquery_notin.q.out       | 378 ++++++++------
 .../clientpositive/llap/subquery_scalar.q.out      | 114 ++--
 .../clientpositive/llap/subquery_select.q.out      | 152 +++---
 .../llap/tez_dynpart_hashjoin_3.q.out              |  60 ++-
 .../llap/tez_fixed_bucket_pruning.q.out            |  25 +-
 .../clientpositive/llap/tez_join_tests.q.out       |  51 +-
 .../clientpositive/llap/tez_joins_explain.q.out    |  51 +-
 .../clientpositive/llap/tez_nway_join.q.out        |  72 ++-
 .../clientpositive/llap/tez_smb_empty.q.out        |  10 +-
 .../clientpositive/llap/tez_smb_reduce_side.q.out  |  28 +-
 .../results/clientpositive/llap/tez_union.q.out    |  72 +--
 .../clientpositive/llap/transitive_not_null.q.out  | 351 +++++++++++++
 .../clientpositive/llap/vector_case_when_1.q.out   |  12 +-
 .../clientpositive/llap/vector_coalesce_3.q.out    |  52 +-
 .../llap/vector_groupby_mapjoin.q.out              |  57 +-
 .../clientpositive/llap/vector_join_nulls.q.out    | 162 +++---
 .../llap/vector_left_outer_join.q.out              |  48 +-
 .../llap/vector_left_outer_join2.q.out             | 260 ++++++----
 .../clientpositive/llap/vector_outer_join0.q.out   |  96 ++--
 .../clientpositive/llap/vector_outer_join1.q.out   | 176 ++++---
 .../clientpositive/llap/vector_outer_join2.q.out   |  84 +--
 .../vectorized_dynamic_partition_pruning.q.out     |   4 +
 .../clientpositive/llap/vectorized_join46.q.out    | 172 +++---
 ql/src/test/results/clientpositive/mapjoin1.q.out  | 132 +++--
 ql/src/test/results/clientpositive/mapjoin3.q.out  |  20 +-
 ql/src/test/results/clientpositive/mapjoin46.q.out | 244 +++++----
 ql/src/test/results/clientpositive/mapjoin47.q.out | 178 ++++---
 .../test/results/clientpositive/masking_10.q.out   |   2 +-
 ql/src/test/results/clientpositive/mergejoin.q.out |  36 +-
 .../results/clientpositive/mergejoins_mixed.q.out  | 196 ++++---
 .../results/clientpositive/optional_outer.q.out    |  80 +--
 .../clientpositive/perf/spark/query40.q.out        |   4 +-
 .../clientpositive/perf/spark/query45.q.out        |   4 +-
 .../results/clientpositive/perf/spark/query5.q.out |   2 +-
 .../clientpositive/perf/spark/query72.q.out        |  22 +-
 .../clientpositive/perf/spark/query75.q.out        |  24 +-
 .../clientpositive/perf/spark/query78.q.out        |  12 +-
 .../clientpositive/perf/spark/query80.q.out        |  12 +-
 .../clientpositive/perf/spark/query93.q.out        |   2 +-
 .../clientpositive/perf/tez/cbo_query40.q.out      |   2 +-
 .../clientpositive/perf/tez/cbo_query5.q.out       |   2 +-
 .../clientpositive/perf/tez/cbo_query72.q.out      |   5 +-
 .../clientpositive/perf/tez/cbo_query75.q.out      |  12 +-
 .../clientpositive/perf/tez/cbo_query78.q.out      |   6 +-
 .../clientpositive/perf/tez/cbo_query80.q.out      |   6 +-
 .../clientpositive/perf/tez/cbo_query93.q.out      |   2 +-
 .../results/clientpositive/perf/tez/query40.q.out  |   2 +-
 .../results/clientpositive/perf/tez/query45.q.out  |   2 +-
 .../results/clientpositive/perf/tez/query72.q.out  | 176 +++----
 .../results/clientpositive/perf/tez/query75.q.out  |   6 +-
 .../results/clientpositive/perf/tez/query78.q.out  |   6 +-
 .../results/clientpositive/perf/tez/query80.q.out  |   6 +-
 .../clientpositive/position_alias_test_1.q.out     |  22 +-
 .../results/clientpositive/skewjoin_mapjoin1.q.out |  10 +-
 .../clientpositive/skewjoin_mapjoin10.q.out        |  10 +-
 .../clientpositive/skewjoin_union_remove_1.q.out   |  10 +-
 .../test/results/clientpositive/skewjoinopt1.q.out |  10 +-
 .../test/results/clientpositive/skewjoinopt2.q.out |  10 +-
 .../results/clientpositive/smb_mapjoin_47.q.out    |  62 +--
 .../clientpositive/spark/annotate_stats_join.q.out |  38 +-
 .../spark/auto_sortmerge_join_16.q.out             |  21 +-
 .../spark/auto_sortmerge_join_16.q.out_spark       |  21 +-
 .../results/clientpositive/spark/innerjoin.q.out   |  20 +-
 .../clientpositive/spark/join32_lessSize.q.out     |  28 +-
 .../spark/join_filters_overlap.q.out               |  66 +--
 .../clientpositive/spark/join_merging.q.out        |  26 +-
 .../results/clientpositive/spark/join_star.q.out   | 162 +++---
 .../results/clientpositive/spark/mapjoin1.q.out    | 132 +++--
 .../clientpositive/spark/mergejoins_mixed.q.out    | 196 ++++---
 .../spark/skewjoin_union_remove_1.q.out            |  10 +-
 .../clientpositive/spark/skewjoinopt1.q.out        |  10 +-
 .../clientpositive/spark/skewjoinopt15.q.out       |  10 +-
 .../clientpositive/spark/skewjoinopt2.q.out        |  10 +-
 .../clientpositive/spark/smb_mapjoin_17.q.out      |  96 ++--
 .../clientpositive/spark/smb_mapjoin_4.q.out       | 139 ++---
 .../clientpositive/spark/smb_mapjoin_5.q.out       | 139 ++---
 .../clientpositive/spark/spark_constprog_dpp.q.out |  11 +-
 .../spark/spark_dynamic_partition_pruning.q.out    |   4 +
 .../clientpositive/spark/spark_explainuser_1.q.out | 150 +++---
 ...park_vectorized_dynamic_partition_pruning.q.out |   4 +
 .../clientpositive/spark/subquery_multi.q.out      |  86 ++-
 .../clientpositive/spark/subquery_notin.q.out      | 274 +++++-----
 .../clientpositive/spark/subquery_scalar.q.out     |  44 +-
 .../clientpositive/spark/subquery_select.q.out     | 130 +++--
 .../clientpositive/spark/tez_join_tests.q.out      |  40 +-
 .../clientpositive/spark/tez_joins_explain.q.out   |  40 +-
 .../spark/vector_left_outer_join.q.out             |  36 +-
 .../clientpositive/spark/vector_outer_join0.q.out  |  64 ++-
 .../clientpositive/spark/vector_outer_join1.q.out  | 128 +++--
 .../clientpositive/spark/vector_outer_join2.q.out  |  64 ++-
 .../clientpositive/subquery_notexists.q.out        |   4 +-
 .../clientpositive/subquery_notin_having.q.out     |  22 +-
 .../tez/hybridgrace_hashjoin_1.q.out               |  48 +-
 .../clientpositive/vector_case_when_1.q.out        |  12 +-
 .../results/clientpositive/vector_coalesce_3.q.out |  26 +-
 .../clientpositive/vector_groupby_mapjoin.q.out    |  36 +-
 .../clientpositive/vector_left_outer_join.q.out    |  44 +-
 .../clientpositive/vector_left_outer_join2.q.out   | 192 ++++---
 .../clientpositive/vector_outer_join0.q.out        |  48 +-
 .../clientpositive/vector_outer_join1.q.out        |  92 ++--
 .../clientpositive/vector_outer_join2.q.out        |  44 +-
 .../clientpositive/vector_outer_join3.q.out        |   6 +-
 .../clientpositive/vector_outer_join4.q.out        |   6 +-
 .../clientpositive/vector_outer_join6.q.out        |   4 +-
 .../clientpositive/vectorized_join46_mr.q.out      | 104 ++--
 161 files changed, 6759 insertions(+), 4690 deletions(-)
 create mode 100644 ql/src/test/queries/clientpositive/fold_varchar.q
 create mode 100644 ql/src/test/queries/clientpositive/transitive_not_null.q
 create mode 100644 ql/src/test/results/clientpositive/llap/fold_varchar.q.out
 create mode 100644 ql/src/test/results/clientpositive/llap/transitive_not_null.q.out


[hive] 01/02: HIVE-21316: Comparision of varchar column and string literal should happen in varchar (Zoltan Haindrich reviewed by Vineet Garg)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 09c0de63c1d4d657cb47b467fb6f370a403eb532
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Mon Apr 1 09:41:48 2019 +0200

    HIVE-21316: Comparision of varchar column and string literal should happen in varchar (Zoltan Haindrich reviewed by Vineet Garg)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../test/resources/testconfiguration.properties    |  1 +
 .../calcite/translator/ExprNodeConverter.java      | 24 ++++++++-
 .../calcite/translator/RexNodeConverter.java       | 26 +++++++--
 .../hadoop/hive/ql/parse/TypeCheckProcFactory.java | 41 ++++++++++-----
 ql/src/test/queries/clientpositive/fold_varchar.q  | 17 ++++++
 .../clientpositive/in_typecheck_varchar.q.out      |  9 ++--
 .../results/clientpositive/llap/fold_varchar.q.out | 61 ++++++++++++++++++++++
 .../llap/materialized_view_rewrite_1.q.out         |  8 +--
 .../llap/materialized_view_rewrite_6.q.out         |  4 +-
 .../materialized_view_rewrite_no_join_opt.q.out    |  4 +-
 .../clientpositive/llap/vector_case_when_1.q.out   | 12 ++---
 .../clientpositive/vector_case_when_1.q.out        | 12 ++---
 12 files changed, 175 insertions(+), 44 deletions(-)

diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 7f79995..ff006b7 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -532,6 +532,7 @@ minillaplocal.query.files=\
   external_jdbc_table_partition.q,\
   external_jdbc_table_typeconversion.q,\
   fullouter_mapjoin_1_optimized.q,\
+  fold_varchar.q,\
   get_splits_0.q,\
   groupby2.q,\
   groupby_groupingset_bug.q,\
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
index 6dd0018..cf104af 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
@@ -44,9 +44,11 @@ import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.TimeString;
 import org.apache.calcite.util.TimestampString;
 import org.apache.hadoop.hive.common.type.Date;
+import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -56,6 +58,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcFactory;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.RexVisitor;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.Schema;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.RexNodeConverter.HiveNlsString;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order;
@@ -77,10 +80,12 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -332,7 +337,24 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
             lType.getScale()), HiveDecimal.create((BigDecimal)literal.getValue3()));
       case VARCHAR:
       case CHAR: {
-        return new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, literal.getValue3());
+        if (literal.getValue() instanceof HiveNlsString) {
+          HiveNlsString mxNlsString = (HiveNlsString) literal.getValue();
+          switch (mxNlsString.interpretation) {
+          case STRING:
+            return new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, literal.getValue3());
+          case CHAR: {
+            int precision = lType.getPrecision();
+            HiveChar value = new HiveChar((String) literal.getValue3(), precision);
+            return new ExprNodeConstantDesc(new CharTypeInfo(precision), value);
+          }
+          case VARCHAR: {
+            int precision = lType.getPrecision();
+            HiveVarchar value = new HiveVarchar((String) literal.getValue3(), precision);
+            return new ExprNodeConstantDesc(new VarcharTypeInfo(precision), value);
+          }
+          }
+        }
+        throw new RuntimeException("varchar/string/char values must use HiveNlsString for correctness");
       }
       case INTERVAL_YEAR:
       case INTERVAL_MONTH:
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
index d15c710..1134cf3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSubquerySemanticExcept
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExtractDate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFloorDate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveToDateSqlOperator;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.RexNodeConverter.HiveNlsString.Interpretation;
 import org.apache.hadoop.hive.ql.parse.ParseUtils;
 import org.apache.hadoop.hive.ql.parse.RowResolver;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -712,8 +713,23 @@ public class RexNodeConverter {
   private static final BigInteger MIN_LONG_BI = BigInteger.valueOf(Long.MIN_VALUE),
       MAX_LONG_BI = BigInteger.valueOf(Long.MAX_VALUE);
 
-  private static NlsString asUnicodeString(String text) {
-    return new NlsString(text, ConversionUtil.NATIVE_UTF16_CHARSET_NAME, SqlCollation.IMPLICIT);
+  private static NlsString makeHiveUnicodeString(Interpretation interpretation, String text) {
+    return new HiveNlsString(interpretation, text, ConversionUtil.NATIVE_UTF16_CHARSET_NAME, SqlCollation.IMPLICIT);
+  }
+
+  static class HiveNlsString extends NlsString {
+
+    enum Interpretation {
+      CHAR, VARCHAR, STRING;
+    }
+
+    public final Interpretation interpretation;
+
+    public HiveNlsString(Interpretation interpretation, String value, String charsetName, SqlCollation collation) {
+      super(value, charsetName, collation);
+      this.interpretation = interpretation;
+    }
+
   }
 
   protected RexNode convert(ExprNodeConstantDesc literal) throws CalciteSemanticException {
@@ -807,16 +823,16 @@ public class RexNodeConverter {
       if (value instanceof HiveChar) {
         value = ((HiveChar) value).getValue();
       }
-      calciteLiteral = rexBuilder.makeCharLiteral(asUnicodeString((String) value));
+      calciteLiteral = rexBuilder.makeCharLiteral(makeHiveUnicodeString(Interpretation.CHAR, (String) value));
       break;
     case VARCHAR:
       if (value instanceof HiveVarchar) {
         value = ((HiveVarchar) value).getValue();
       }
-      calciteLiteral = rexBuilder.makeCharLiteral(asUnicodeString((String) value));
+      calciteLiteral = rexBuilder.makeCharLiteral(makeHiveUnicodeString(Interpretation.VARCHAR, (String) value));
       break;
     case STRING:
-      calciteLiteral = rexBuilder.makeCharLiteral(asUnicodeString((String) value));
+      calciteLiteral = rexBuilder.makeCharLiteral(makeHiveUnicodeString(Interpretation.STRING, (String) value));
       break;
     case DATE:
       final Date date = (Date) value;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 357439a..dff108a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.common.type.Timestamp;
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -1424,22 +1425,36 @@ public class TypeCheckProcFactory {
         return hiveDecimal;
       }
 
-      // TODO : Char and string comparison happens in char. But, varchar and string comparison happens in String.
-
-      // if column type is char and constant type is string, then convert the constant to char
-      // type with padded spaces.
       String constTypeInfoName = constTypeInfo.getTypeName();
-      if (constTypeInfoName.equalsIgnoreCase(serdeConstants.STRING_TYPE_NAME) && colTypeInfo instanceof CharTypeInfo) {
-        final String constValue = constVal.toString();
-        final int length = TypeInfoUtils.getCharacterLengthForType(colTypeInfo);
-        HiveChar newValue = new HiveChar(constValue, length);
-        HiveChar maxCharConst = new HiveChar(constValue, HiveChar.MAX_CHAR_LENGTH);
-        if (maxCharConst.equals(newValue)) {
-          return newValue;
-        } else {
-          return null;
+      if (constTypeInfoName.equalsIgnoreCase(serdeConstants.STRING_TYPE_NAME)) {
+        // because a comparison against a "string" will happen in "string" type.
+        // to avoid unintnetional comparisions in "string"
+        // constants which are representing char/varchar values must be converted to the
+        // appropriate type.
+        if (colTypeInfo instanceof CharTypeInfo) {
+          final String constValue = constVal.toString();
+          final int length = TypeInfoUtils.getCharacterLengthForType(colTypeInfo);
+          HiveChar newValue = new HiveChar(constValue, length);
+          HiveChar maxCharConst = new HiveChar(constValue, HiveChar.MAX_CHAR_LENGTH);
+          if (maxCharConst.equals(newValue)) {
+            return newValue;
+          } else {
+            return null;
+          }
+        }
+        if (colTypeInfo instanceof VarcharTypeInfo) {
+          final String constValue = constVal.toString();
+          final int length = TypeInfoUtils.getCharacterLengthForType(colTypeInfo);
+          HiveVarchar newValue = new HiveVarchar(constValue, length);
+          HiveVarchar maxCharConst = new HiveVarchar(constValue, HiveVarchar.MAX_VARCHAR_LENGTH);
+          if (maxCharConst.equals(newValue)) {
+            return newValue;
+          } else {
+            return null;
+          }
         }
       }
+
       return constVal;
     }
 
diff --git a/ql/src/test/queries/clientpositive/fold_varchar.q b/ql/src/test/queries/clientpositive/fold_varchar.q
new file mode 100644
index 0000000..780fcf1
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/fold_varchar.q
@@ -0,0 +1,17 @@
+
+create table t (a string,vc varchar(10),c char(10));
+insert into t values('bee','bee','bee'),('xxx','xxx','xxx');
+
+select	assert_true(t0.v = t1.v) from
+	(select hash(a) as v from t where a='bee') as t0
+join	(select hash(a) as v from t where a='bee' or a='xbee') as t1 on (true);
+
+select	assert_true(t0.v = t1.v) from
+	(select hash(vc) as v from t where vc='bee') as t0
+join	(select hash(vc) as v from t where vc='bee' or vc='xbee') as t1 on (true);
+
+select	assert_true(t0.v = t1.v) from
+	(select hash(c) as v from t where c='bee') as t0
+join	(select hash(c) as v from t where c='bee' or c='xbee') as t1 on (true);
+
+
diff --git a/ql/src/test/results/clientpositive/in_typecheck_varchar.q.out b/ql/src/test/results/clientpositive/in_typecheck_varchar.q.out
index 0c2f366..cebfe53 100644
--- a/ql/src/test/results/clientpositive/in_typecheck_varchar.q.out
+++ b/ql/src/test/results/clientpositive/in_typecheck_varchar.q.out
@@ -36,10 +36,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ax
-            filterExpr: ((CAST( s AS STRING) = 'a') and (CAST( t AS STRING) = 'a')) (type: boolean)
+            filterExpr: ((s = 'a') and (t = 'a')) (type: boolean)
             Statistics: Num rows: 3 Data size: 513 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: ((CAST( s AS STRING) = 'a') and (CAST( t AS STRING) = 'a')) (type: boolean)
+              predicate: ((s = 'a') and (t = 'a')) (type: boolean)
               Statistics: Num rows: 1 Data size: 171 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 Statistics: Num rows: 1 Data size: 171 Basic stats: COMPLETE Column stats: COMPLETE
@@ -107,10 +107,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: ax
-            filterExpr: (struct(CAST( s AS STRING),CAST( t AS STRING))) IN (const struct('a','a'), const struct('b','bb')) (type: boolean)
+            filterExpr: (struct(s,t)) IN (const struct('a','a'), const struct('b','bb')) (type: boolean)
             Statistics: Num rows: 3 Data size: 513 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (struct(CAST( s AS STRING),CAST( t AS STRING))) IN (const struct('a','a'), const struct('b','bb')) (type: boolean)
+              predicate: (struct(s,t)) IN (const struct('a','a'), const struct('b','bb')) (type: boolean)
               Statistics: Num rows: 1 Data size: 171 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 Statistics: Num rows: 1 Data size: 171 Basic stats: COMPLETE Column stats: COMPLETE
@@ -124,7 +124,6 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: bigint)
-      Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
diff --git a/ql/src/test/results/clientpositive/llap/fold_varchar.q.out b/ql/src/test/results/clientpositive/llap/fold_varchar.q.out
new file mode 100644
index 0000000..fe8e0da
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/fold_varchar.q.out
@@ -0,0 +1,61 @@
+PREHOOK: query: create table t (a string,vc varchar(10),c char(10))
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t
+POSTHOOK: query: create table t (a string,vc varchar(10),c char(10))
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t
+PREHOOK: query: insert into t values('bee','bee','bee'),('xxx','xxx','xxx')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@t
+POSTHOOK: query: insert into t values('bee','bee','bee'),('xxx','xxx','xxx')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@t
+POSTHOOK: Lineage: t.a SCRIPT []
+POSTHOOK: Lineage: t.c SCRIPT []
+POSTHOOK: Lineage: t.vc SCRIPT []
+Warning: Shuffle Join MERGEJOIN[13][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: select	assert_true(t0.v = t1.v) from
+	(select hash(a) as v from t where a='bee') as t0
+join	(select hash(a) as v from t where a='bee' or a='xbee') as t1 on (true)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: select	assert_true(t0.v = t1.v) from
+	(select hash(a) as v from t where a='bee') as t0
+join	(select hash(a) as v from t where a='bee' or a='xbee') as t1 on (true)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+NULL
+Warning: Shuffle Join MERGEJOIN[13][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: select	assert_true(t0.v = t1.v) from
+	(select hash(vc) as v from t where vc='bee') as t0
+join	(select hash(vc) as v from t where vc='bee' or vc='xbee') as t1 on (true)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: select	assert_true(t0.v = t1.v) from
+	(select hash(vc) as v from t where vc='bee') as t0
+join	(select hash(vc) as v from t where vc='bee' or vc='xbee') as t1 on (true)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+NULL
+Warning: Shuffle Join MERGEJOIN[13][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: select	assert_true(t0.v = t1.v) from
+	(select hash(c) as v from t where c='bee') as t0
+join	(select hash(c) as v from t where c='bee' or c='xbee') as t1 on (true)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: select	assert_true(t0.v = t1.v) from
+	(select hash(c) as v from t where c='bee') as t0
+join	(select hash(c) as v from t where c='bee' or c='xbee') as t1 on (true)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+NULL
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_1.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_1.q.out
index b405c08..c0f9a15 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_1.q.out
@@ -1069,10 +1069,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: default.mv1_n2
-                  filterExpr: (CAST( name AS STRING) = 'Sales') (type: boolean)
+                  filterExpr: (name = 'Sales') (type: boolean)
                   Statistics: Num rows: 3 Data size: 291 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (CAST( name AS STRING) = 'Sales') (type: boolean)
+                    predicate: (name = 'Sales') (type: boolean)
                     Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c (type: bigint)
@@ -1195,10 +1195,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: default.mv1_n2
-                  filterExpr: (CAST( name AS STRING) = 'Sales') (type: boolean)
+                  filterExpr: (name = 'Sales') (type: boolean)
                   Statistics: Num rows: 3 Data size: 303 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (CAST( name AS STRING) = 'Sales') (type: boolean)
+                    predicate: (name = 'Sales') (type: boolean)
                     Statistics: Num rows: 1 Data size: 101 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: deptno (type: int), c (type: bigint)
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_6.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_6.q.out
index 6c404f1..b2a1708 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_6.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_6.q.out
@@ -775,9 +775,9 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: default.mv1
-          filterExpr: (CAST( name1 AS STRING) = 'Bill') (type: boolean)
+          filterExpr: (name1 = 'Bill') (type: boolean)
           Filter Operator
-            predicate: (CAST( name1 AS STRING) = 'Bill') (type: boolean)
+            predicate: (name1 = 'Bill') (type: boolean)
             Select Operator
               expressions: empid (type: int), empid (type: int), deptno (type: int)
               outputColumnNames: _col0, _col1, _col2
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_no_join_opt.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_no_join_opt.q.out
index 36a2b4d..4845755 100644
--- a/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_no_join_opt.q.out
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_rewrite_no_join_opt.q.out
@@ -639,10 +639,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: default.mv1_n20
-                  filterExpr: ((CAST( name1 AS STRING) = 'Bill') and deptno is not null) (type: boolean)
+                  filterExpr: ((name1 = 'Bill') and deptno is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: ((CAST( name1 AS STRING) = 'Bill') and deptno is not null) (type: boolean)
+                    predicate: ((name1 = 'Bill') and deptno is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: empid (type: int), deptno (type: int)
diff --git a/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out b/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
index 08200b4..5323159 100644
--- a/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_case_when_1.q.out
@@ -209,13 +209,13 @@ STAGE PLANS:
                       native: true
                       vectorizationSchemaColumns: [0:l_orderkey:int, 1:l_partkey:int, 2:l_suppkey:int, 3:l_linenumber:int, 4:l_quantity:int, 5:l_extendedprice:double, 6:l_discount:double, 7:l_tax:decimal(10,2)/DECIMAL_64, 8:l_returnflag:char(1), 9:l_linestatus:char(1), 10:l_shipdate:date, 11:l_commitdate:date, 12:l_receiptdate:date, 13:l_shipinstruct:varchar(20), 14:l_shipmode:char(10), 15:l_comment:string, 16:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
                   Select Operator
-                    expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Sing [...]
+                    expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Sing [...]
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [4, 21, 26, 30, 34, 38, 42, 44, 46, 48, 50, 52, 54, 58, 61, 64, 67]
-                        selectExpressions: VectorUDFAdaptor(CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:stri [...]
+                        selectExpressions: VectorUDFAdaptor(CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:stri [...]
                     Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
@@ -546,13 +546,13 @@ STAGE PLANS:
                       native: true
                       vectorizationSchemaColumns: [0:l_orderkey:int, 1:l_partkey:int, 2:l_suppkey:int, 3:l_linenumber:int, 4:l_quantity:int, 5:l_extendedprice:double, 6:l_discount:double, 7:l_tax:decimal(10,2)/DECIMAL_64, 8:l_returnflag:char(1), 9:l_linestatus:char(1), 10:l_shipdate:date, 11:l_commitdate:date, 12:l_receiptdate:date, 13:l_shipinstruct:varchar(20), 14:l_shipmode:char(10), 15:l_comment:string, 16:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
                   Select Operator
-                    expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Sing [...]
+                    expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Sing [...]
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [4, 24, 33, 40, 44, 49, 53, 55, 57, 59, 61, 63, 65, 69, 72, 75, 78]
-                        selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringS [...]
+                        selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringS [...]
                     Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
@@ -883,13 +883,13 @@ STAGE PLANS:
                       native: true
                       vectorizationSchemaColumns: [0:l_orderkey:int, 1:l_partkey:int, 2:l_suppkey:int, 3:l_linenumber:int, 4:l_quantity:int, 5:l_extendedprice:double, 6:l_discount:double, 7:l_tax:decimal(10,2)/DECIMAL_64, 8:l_returnflag:char(1), 9:l_linestatus:char(1), 10:l_shipdate:date, 11:l_commitdate:date, 12:l_receiptdate:date, 13:l_shipinstruct:varchar(20), 14:l_shipmode:char(10), 15:l_comment:string, 16:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
                   Select Operator
-                    expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Sing [...]
+                    expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Sing [...]
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                     Select Vectorization:
                         className: VectorSelectOperator
                         native: true
                         projectedOutputColumnNums: [4, 27, 39, 48, 52, 57, 62, 64, 66, 71, 76, 78, 80, 84, 87, 90, 93]
-                        selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLo [...]
+                        selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLo [...]
                     Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
diff --git a/ql/src/test/results/clientpositive/vector_case_when_1.q.out b/ql/src/test/results/clientpositive/vector_case_when_1.q.out
index 217af67..bedfd54 100644
--- a/ql/src/test/results/clientpositive/vector_case_when_1.q.out
+++ b/ql/src/test/results/clientpositive/vector_case_when_1.q.out
@@ -206,13 +206,13 @@ STAGE PLANS:
                 native: true
                 vectorizationSchemaColumns: [0:l_orderkey:int, 1:l_partkey:int, 2:l_suppkey:int, 3:l_linenumber:int, 4:l_quantity:int, 5:l_extendedprice:double, 6:l_discount:double, 7:l_tax:decimal(10,2)/DECIMAL_64, 8:l_returnflag:char(1), 9:l_linestatus:char(1), 10:l_shipdate:date, 11:l_commitdate:date, 12:l_receiptdate:date, 13:l_shipinstruct:varchar(20), 14:l_shipmode:char(10), 15:l_comment:string, 16:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
             Select Operator
-              expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') W [...]
+              expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') W [...]
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [4, 21, 26, 30, 34, 38, 42, 44, 46, 48, 50, 52, 54, 58, 61, 64, 67]
-                  selectExpressions: VectorUDFAdaptor(CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:string, Ve [...]
+                  selectExpressions: VectorUDFAdaptor(CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, LongColLessLongScalar(col 4:int, val 100) -> 20:boolean) -> 21:string, Ve [...]
               Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
@@ -539,13 +539,13 @@ STAGE PLANS:
                 native: true
                 vectorizationSchemaColumns: [0:l_orderkey:int, 1:l_partkey:int, 2:l_suppkey:int, 3:l_linenumber:int, 4:l_quantity:int, 5:l_extendedprice:double, 6:l_discount:double, 7:l_tax:decimal(10,2)/DECIMAL_64, 8:l_returnflag:char(1), 9:l_linestatus:char(1), 10:l_shipdate:date, 11:l_commitdate:date, 12:l_receiptdate:date, 13:l_shipinstruct:varchar(20), 14:l_shipmode:char(10), 15:l_comment:string, 16:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
             Select Operator
-              expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') W [...]
+              expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') W [...]
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [4, 24, 33, 40, 44, 49, 53, 55, 57, 59, 61, 63, 65, 69, 72, 75, 78]
-                  selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringScalar( [...]
+                  selectExpressions: IfExprStringScalarStringGroupColumn(col 17:boolean, val Singlecol 23:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, IfExprStringScalarStringGroupColumn(col 18:boolean, val Twocol 22:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 18:boolean, IfExprStringScalarStringGroupColumn(col 19:boolean, val Somecol 21:string)(children: LongColLessLongScalar(col 4:int, val 10) -> 19:boolean, IfExprStringScalarStringScalar( [...]
               Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false
@@ -872,13 +872,13 @@ STAGE PLANS:
                 native: true
                 vectorizationSchemaColumns: [0:l_orderkey:int, 1:l_partkey:int, 2:l_suppkey:int, 3:l_linenumber:int, 4:l_quantity:int, 5:l_extendedprice:double, 6:l_discount:double, 7:l_tax:decimal(10,2)/DECIMAL_64, 8:l_returnflag:char(1), 9:l_linestatus:char(1), 10:l_shipdate:date, 11:l_commitdate:date, 12:l_receiptdate:date, 13:l_shipinstruct:varchar(20), 14:l_shipmode:char(10), 15:l_comment:string, 16:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
             Select Operator
-              expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') W [...]
+              expressions: l_quantity (type: int), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE ('Huge number') END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') WHEN ((l_quantity = 2)) THEN ('Two') WHEN ((l_quantity < 10)) THEN ('Some') WHEN ((l_quantity < 100)) THEN ('Many') ELSE (null) END (type: string), CASE WHEN ((l_quantity = 1)) THEN ('Single') W [...]
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
               Select Vectorization:
                   className: VectorSelectOperator
                   native: true
                   projectedOutputColumnNums: [4, 27, 39, 48, 52, 57, 62, 64, 66, 71, 76, 78, 80, 84, 87, 90, 93]
-                  selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLongScal [...]
+                  selectExpressions: IfExprColumnCondExpr(col 17:boolean, col 18:stringcol 26:string)(children: LongColEqualLongScalar(col 4:int, val 1) -> 17:boolean, ConstantVectorExpression(val Single) -> 18:string, IfExprColumnCondExpr(col 19:boolean, col 20:stringcol 25:string)(children: LongColEqualLongScalar(col 4:int, val 2) -> 19:boolean, ConstantVectorExpression(val Two) -> 20:string, IfExprColumnCondExpr(col 21:boolean, col 22:stringcol 24:string)(children: LongColLessLongScal [...]
               Statistics: Num rows: 101 Data size: 141804 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
                 compressed: false


[hive] 02/02: HIVE-21230: LEFT OUTER JOIN does not generate transitive IS NOT NULL filter on right side (HiveJoinAddNotNullRule bails out for outer joins) (Vineet Garg via Jesus Camacho Rodriguez)

Posted by kg...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit f8a73a8a51a4e4197f2b5a7d8c861dc543b32971
Author: Vineet Garg <vg...@apache.org>
AuthorDate: Mon Apr 1 09:43:48 2019 +0200

    HIVE-21230: LEFT OUTER JOIN does not generate transitive IS NOT NULL filter on right side (HiveJoinAddNotNullRule bails out for outer joins) (Vineet Garg via Jesus Camacho Rodriguez)
    
    Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
---
 .../test/resources/testconfiguration.properties    |   1 +
 .../calcite/rules/HiveJoinAddNotNullRule.java      |  83 ++-
 .../queries/clientpositive/transitive_not_null.q   |  53 ++
 .../clientpositive/annotate_stats_join.q.out       |  38 +-
 .../cbo_SortUnionTransposeRule.q.out               |  80 +--
 .../test/results/clientpositive/cbo_rp_join0.q.out |  30 +-
 .../results/clientpositive/constant_prop_3.q.out   |   4 +-
 .../clientpositive/correlationoptimizer8.q.out     |  40 +-
 .../results/clientpositive/infer_join_preds.q.out  |  40 +-
 ql/src/test/results/clientpositive/innerjoin.q.out |  20 +-
 ql/src/test/results/clientpositive/join45.q.out    | 144 +++---
 ql/src/test/results/clientpositive/join46.q.out    | 200 +++----
 ql/src/test/results/clientpositive/join47.q.out    | 142 ++---
 .../join_cond_pushdown_unqual5.q.out               |  38 +-
 .../clientpositive/join_emit_interval.q.out        |  30 +-
 .../clientpositive/join_filters_overlap.q.out      |  66 +--
 .../test/results/clientpositive/join_merging.q.out |  38 +-
 ql/src/test/results/clientpositive/join_star.q.out | 162 +++---
 ql/src/test/results/clientpositive/lineage1.q.out  |  40 +-
 .../llap/auto_sortmerge_join_14.q.out              |  20 +-
 .../llap/auto_sortmerge_join_15.q.out              |  20 +-
 .../llap/auto_sortmerge_join_16.q.out              |  69 +--
 .../clientpositive/llap/check_constraint.q.out     |  22 +-
 .../clientpositive/llap/constprog_dpp.q.out        |  54 +-
 .../llap/correlationoptimizer1.q.out               | 180 ++++---
 .../llap/correlationoptimizer2.q.out               | 112 ++--
 .../llap/correlationoptimizer4.q.out               | 216 +++++---
 .../llap/dynamic_partition_pruning.q.out           |   4 +
 .../clientpositive/llap/explainuser_1.q.out        | 140 ++---
 .../llap/hybridgrace_hashjoin_1.q.out              |  48 +-
 .../clientpositive/llap/join32_lessSize.q.out      |  28 +-
 .../test/results/clientpositive/llap/join46.q.out  | 212 ++++----
 .../clientpositive/llap/join_emit_interval.q.out   |  30 +-
 .../clientpositive/llap/limit_join_transpose.q.out | 344 +++++++-----
 .../results/clientpositive/llap/lineage2.q.out     |   2 +-
 .../results/clientpositive/llap/lineage3.q.out     |   4 +-
 .../results/clientpositive/llap/mapjoin3.q.out     |  22 +-
 .../results/clientpositive/llap/mapjoin46.q.out    | 322 ++++++------
 .../llap/mapjoin_emit_interval.q.out               |  30 +-
 .../results/clientpositive/llap/mergejoin.q.out    |  20 +-
 .../results/clientpositive/llap/sharedwork.q.out   |  58 ++-
 .../clientpositive/llap/skewjoinopt15.q.out        |  38 +-
 .../clientpositive/llap/smb_mapjoin_17.q.out       | 576 +++++++++++++--------
 .../clientpositive/llap/smb_mapjoin_4.q.out        | 264 +++++-----
 .../clientpositive/llap/smb_mapjoin_5.q.out        | 264 +++++-----
 .../clientpositive/llap/subquery_in_having.q.out   |  70 +--
 .../clientpositive/llap/subquery_multi.q.out       | 125 +++--
 .../clientpositive/llap/subquery_notin.q.out       | 378 ++++++++------
 .../clientpositive/llap/subquery_scalar.q.out      | 114 ++--
 .../clientpositive/llap/subquery_select.q.out      | 152 +++---
 .../llap/tez_dynpart_hashjoin_3.q.out              |  60 ++-
 .../llap/tez_fixed_bucket_pruning.q.out            |  25 +-
 .../clientpositive/llap/tez_join_tests.q.out       |  51 +-
 .../clientpositive/llap/tez_joins_explain.q.out    |  51 +-
 .../clientpositive/llap/tez_nway_join.q.out        |  72 ++-
 .../clientpositive/llap/tez_smb_empty.q.out        |  10 +-
 .../clientpositive/llap/tez_smb_reduce_side.q.out  |  28 +-
 .../results/clientpositive/llap/tez_union.q.out    |  72 +--
 .../clientpositive/llap/transitive_not_null.q.out  | 351 +++++++++++++
 .../clientpositive/llap/vector_coalesce_3.q.out    |  52 +-
 .../llap/vector_groupby_mapjoin.q.out              |  57 +-
 .../clientpositive/llap/vector_join_nulls.q.out    | 162 +++---
 .../llap/vector_left_outer_join.q.out              |  48 +-
 .../llap/vector_left_outer_join2.q.out             | 260 ++++++----
 .../clientpositive/llap/vector_outer_join0.q.out   |  96 ++--
 .../clientpositive/llap/vector_outer_join1.q.out   | 176 ++++---
 .../clientpositive/llap/vector_outer_join2.q.out   |  84 +--
 .../vectorized_dynamic_partition_pruning.q.out     |   4 +
 .../clientpositive/llap/vectorized_join46.q.out    | 172 +++---
 ql/src/test/results/clientpositive/mapjoin1.q.out  | 132 +++--
 ql/src/test/results/clientpositive/mapjoin3.q.out  |  20 +-
 ql/src/test/results/clientpositive/mapjoin46.q.out | 244 +++++----
 ql/src/test/results/clientpositive/mapjoin47.q.out | 178 ++++---
 .../test/results/clientpositive/masking_10.q.out   |   2 +-
 ql/src/test/results/clientpositive/mergejoin.q.out |  36 +-
 .../results/clientpositive/mergejoins_mixed.q.out  | 196 ++++---
 .../results/clientpositive/optional_outer.q.out    |  80 +--
 .../clientpositive/perf/spark/query40.q.out        |   4 +-
 .../clientpositive/perf/spark/query45.q.out        |   4 +-
 .../results/clientpositive/perf/spark/query5.q.out |   2 +-
 .../clientpositive/perf/spark/query72.q.out        |  22 +-
 .../clientpositive/perf/spark/query75.q.out        |  24 +-
 .../clientpositive/perf/spark/query78.q.out        |  12 +-
 .../clientpositive/perf/spark/query80.q.out        |  12 +-
 .../clientpositive/perf/spark/query93.q.out        |   2 +-
 .../clientpositive/perf/tez/cbo_query40.q.out      |   2 +-
 .../clientpositive/perf/tez/cbo_query5.q.out       |   2 +-
 .../clientpositive/perf/tez/cbo_query72.q.out      |   5 +-
 .../clientpositive/perf/tez/cbo_query75.q.out      |  12 +-
 .../clientpositive/perf/tez/cbo_query78.q.out      |   6 +-
 .../clientpositive/perf/tez/cbo_query80.q.out      |   6 +-
 .../clientpositive/perf/tez/cbo_query93.q.out      |   2 +-
 .../results/clientpositive/perf/tez/query40.q.out  |   2 +-
 .../results/clientpositive/perf/tez/query45.q.out  |   2 +-
 .../results/clientpositive/perf/tez/query72.q.out  | 176 +++----
 .../results/clientpositive/perf/tez/query75.q.out  |   6 +-
 .../results/clientpositive/perf/tez/query78.q.out  |   6 +-
 .../results/clientpositive/perf/tez/query80.q.out  |   6 +-
 .../clientpositive/position_alias_test_1.q.out     |  22 +-
 .../results/clientpositive/skewjoin_mapjoin1.q.out |  10 +-
 .../clientpositive/skewjoin_mapjoin10.q.out        |  10 +-
 .../clientpositive/skewjoin_union_remove_1.q.out   |  10 +-
 .../test/results/clientpositive/skewjoinopt1.q.out |  10 +-
 .../test/results/clientpositive/skewjoinopt2.q.out |  10 +-
 .../results/clientpositive/smb_mapjoin_47.q.out    |  62 +--
 .../clientpositive/spark/annotate_stats_join.q.out |  38 +-
 .../spark/auto_sortmerge_join_16.q.out             |  21 +-
 .../spark/auto_sortmerge_join_16.q.out_spark       |  21 +-
 .../results/clientpositive/spark/innerjoin.q.out   |  20 +-
 .../clientpositive/spark/join32_lessSize.q.out     |  28 +-
 .../spark/join_filters_overlap.q.out               |  66 +--
 .../clientpositive/spark/join_merging.q.out        |  26 +-
 .../results/clientpositive/spark/join_star.q.out   | 162 +++---
 .../results/clientpositive/spark/mapjoin1.q.out    | 132 +++--
 .../clientpositive/spark/mergejoins_mixed.q.out    | 196 ++++---
 .../spark/skewjoin_union_remove_1.q.out            |  10 +-
 .../clientpositive/spark/skewjoinopt1.q.out        |  10 +-
 .../clientpositive/spark/skewjoinopt15.q.out       |  10 +-
 .../clientpositive/spark/skewjoinopt2.q.out        |  10 +-
 .../clientpositive/spark/smb_mapjoin_17.q.out      |  96 ++--
 .../clientpositive/spark/smb_mapjoin_4.q.out       | 139 ++---
 .../clientpositive/spark/smb_mapjoin_5.q.out       | 139 ++---
 .../clientpositive/spark/spark_constprog_dpp.q.out |  11 +-
 .../spark/spark_dynamic_partition_pruning.q.out    |   4 +
 .../clientpositive/spark/spark_explainuser_1.q.out | 150 +++---
 ...park_vectorized_dynamic_partition_pruning.q.out |   4 +
 .../clientpositive/spark/subquery_multi.q.out      |  86 ++-
 .../clientpositive/spark/subquery_notin.q.out      | 274 +++++-----
 .../clientpositive/spark/subquery_scalar.q.out     |  44 +-
 .../clientpositive/spark/subquery_select.q.out     | 130 +++--
 .../clientpositive/spark/tez_join_tests.q.out      |  40 +-
 .../clientpositive/spark/tez_joins_explain.q.out   |  40 +-
 .../spark/vector_left_outer_join.q.out             |  36 +-
 .../clientpositive/spark/vector_outer_join0.q.out  |  64 ++-
 .../clientpositive/spark/vector_outer_join1.q.out  | 128 +++--
 .../clientpositive/spark/vector_outer_join2.q.out  |  64 ++-
 .../clientpositive/subquery_notexists.q.out        |   4 +-
 .../clientpositive/subquery_notin_having.q.out     |  22 +-
 .../tez/hybridgrace_hashjoin_1.q.out               |  48 +-
 .../results/clientpositive/vector_coalesce_3.q.out |  26 +-
 .../clientpositive/vector_groupby_mapjoin.q.out    |  36 +-
 .../clientpositive/vector_left_outer_join.q.out    |  44 +-
 .../clientpositive/vector_left_outer_join2.q.out   | 192 ++++---
 .../clientpositive/vector_outer_join0.q.out        |  48 +-
 .../clientpositive/vector_outer_join1.q.out        |  92 ++--
 .../clientpositive/vector_outer_join2.q.out        |  44 +-
 .../clientpositive/vector_outer_join3.q.out        |   6 +-
 .../clientpositive/vector_outer_join4.q.out        |   6 +-
 .../clientpositive/vector_outer_join6.q.out        |   4 +-
 .../clientpositive/vectorized_join46_mr.q.out      | 104 ++--
 150 files changed, 6584 insertions(+), 4646 deletions(-)

diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index ff006b7..c61c430 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -770,6 +770,7 @@ minillaplocal.query.files=\
   tez_union_multiinsert.q,\
   tez_vector_dynpart_hashjoin_1.q,\
   tez_vector_dynpart_hashjoin_2.q,\
+  transitive_not_null.q,\
   truncate_external_force.q,\
   uber_reduce.q,\
   udaf_collect_set_2.q,\
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
index 9711625..48e2cc6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
@@ -26,7 +26,6 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.RelFactories.FilterFactory;
 import org.apache.calcite.rex.RexBuilder;
@@ -47,10 +46,10 @@ import com.google.common.collect.Sets;
 public final class HiveJoinAddNotNullRule extends RelOptRule {
 
   public static final HiveJoinAddNotNullRule INSTANCE_JOIN =
-          new HiveJoinAddNotNullRule(HiveJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
+      new HiveJoinAddNotNullRule(HiveJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
 
   public static final HiveJoinAddNotNullRule INSTANCE_SEMIJOIN =
-          new HiveJoinAddNotNullRule(HiveSemiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
+      new HiveJoinAddNotNullRule(HiveSemiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
 
   private final FilterFactory filterFactory;
 
@@ -60,7 +59,7 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
    * Creates an HiveJoinAddNotNullRule.
    */
   public HiveJoinAddNotNullRule(Class<? extends Join> clazz,
-          RelFactories.FilterFactory filterFactory) {
+                                RelFactories.FilterFactory filterFactory) {
     super(operand(clazz, any()));
     this.filterFactory = filterFactory;
   }
@@ -76,11 +75,27 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
     HiveRulesRegistry registry = call.getPlanner().getContext().unwrap(HiveRulesRegistry.class);
     assert registry != null;
 
-    if (join.getJoinType() != JoinRelType.INNER) {
+
+    if (join.getCondition().isAlwaysTrue()) {
       return;
     }
 
-    if (join.getCondition().isAlwaysTrue()) {
+    boolean genPredOnLeftSide = false, genPredOnRightSide = false;
+
+    switch (join.getJoinType()) {
+    case INNER:
+      genPredOnLeftSide = true;
+      genPredOnRightSide = true;
+      break;
+    case LEFT:
+      genPredOnLeftSide = false;
+      genPredOnRightSide = true;
+      break;
+    case RIGHT:
+      genPredOnLeftSide = true;
+      genPredOnRightSide = false;
+      break;
+    default:
       return;
     }
 
@@ -90,45 +105,58 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
     } catch (CalciteSemanticException e) {
       return;
     }
-    
+
     List<RexNode> leftJoinExprsList = new ArrayList<>();
     List<RexNode> rightJoinExprsList = new ArrayList<>();
     for (JoinLeafPredicateInfo joinLeafPredicateInfo : joinPredInfo.getEquiJoinPredicateElements()) {
-        leftJoinExprsList.addAll(joinLeafPredicateInfo.getJoinExprs(0));
-        rightJoinExprsList.addAll(joinLeafPredicateInfo.getJoinExprs(1));
+      leftJoinExprsList.addAll(joinLeafPredicateInfo.getJoinExprs(0));
+      rightJoinExprsList.addAll(joinLeafPredicateInfo.getJoinExprs(1));
     }
 
     // Build not null conditions
     final RelOptCluster cluster = join.getCluster();
     final RexBuilder rexBuilder = join.getCluster().getRexBuilder();
 
+
+    RexNode newLeftPredicate = rexBuilder.makeLiteral(true);
+    RexNode newRightPredicate = rexBuilder.makeLiteral(true);
+
+    // generate predicate on left side
     Set<String> leftPushedPredicates = Sets.newHashSet(registry.getPushedPredicates(join, 0));
-    final List<RexNode> newLeftConditions = getNotNullConditions(cluster,
-            rexBuilder, leftJoinExprsList, leftPushedPredicates);
+    if (genPredOnLeftSide) {
+      List<RexNode> newLeftConditions = getNotNullConditions(cluster,
+                                                             rexBuilder, leftJoinExprsList, leftPushedPredicates);
+      newLeftPredicate = RexUtil.composeConjunction(rexBuilder, newLeftConditions, false);
+    }
+
+    // generate predicate on right side
     Set<String> rightPushedPredicates = Sets.newHashSet(registry.getPushedPredicates(join, 1));
-    final List<RexNode> newRightConditions = getNotNullConditions(cluster,
-            rexBuilder, rightJoinExprsList, rightPushedPredicates);
+    if (genPredOnRightSide) {
+      List<RexNode> newRightConditions = getNotNullConditions(cluster,
+                                                              rexBuilder, rightJoinExprsList, rightPushedPredicates);
+      newRightPredicate = RexUtil.composeConjunction(rexBuilder, newRightConditions, false);
+    }
 
-    // Nothing will be added to the expression
-    RexNode newLeftPredicate = RexUtil.composeConjunction(rexBuilder, newLeftConditions, false);
-    RexNode newRightPredicate = RexUtil.composeConjunction(rexBuilder, newRightConditions, false);
-    if (newLeftPredicate.isAlwaysTrue() && newRightPredicate.isAlwaysTrue()) {
+    boolean isNewLeftPredicateAlwaysTrue = newLeftPredicate.isAlwaysTrue();
+    boolean isNewRightPredAlwaysTrue = newRightPredicate.isAlwaysTrue();
+
+    if (isNewLeftPredicateAlwaysTrue && isNewRightPredAlwaysTrue) {
       return;
     }
 
-    if (!newLeftPredicate.isAlwaysTrue()) {
+    if (!isNewLeftPredicateAlwaysTrue) {
       RelNode curr = lChild;
       lChild = filterFactory.createFilter(lChild, newLeftPredicate);
       call.getPlanner().onCopy(curr, lChild);
     }
-    if (!newRightPredicate.isAlwaysTrue()) {
+    if (!isNewRightPredAlwaysTrue) {
       RelNode curr = rChild;
       rChild = filterFactory.createFilter(rChild, newRightPredicate);
       call.getPlanner().onCopy(curr, rChild);
     }
 
     Join newJoin = join.copy(join.getTraitSet(), join.getCondition(),
-            lChild, rChild, join.getJoinType(), join.isSemiJoinDone());
+                             lChild, rChild, join.getJoinType(), join.isSemiJoinDone());
     call.getPlanner().onCopy(join, newJoin);
 
     // Register information about created predicates
@@ -139,18 +167,17 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
   }
 
   private static List<RexNode> getNotNullConditions(RelOptCluster cluster,
-          RexBuilder rexBuilder, List<RexNode> inputJoinExprs,
-          Set<String> pushedPredicates) {
+                                                    RexBuilder rexBuilder, List<RexNode> inputJoinExprs,
+                                                    Set<String> pushedPredicates) {
     final List<RexNode> newConditions = Lists.newArrayList();
 
     for (RexNode rexNode : inputJoinExprs) {
-        RexNode cond = rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, rexNode);
-        String digest = cond.toString();
-        if (pushedPredicates.add(digest)) {
-            newConditions.add(cond);
-        }
+      RexNode cond = rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, rexNode);
+      String digest = cond.toString();
+      if (pushedPredicates.add(digest)) {
+        newConditions.add(cond);
+      }
     }
     return newConditions;
   }
-
 }
diff --git a/ql/src/test/queries/clientpositive/transitive_not_null.q b/ql/src/test/queries/clientpositive/transitive_not_null.q
new file mode 100644
index 0000000..e5fb710
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/transitive_not_null.q
@@ -0,0 +1,53 @@
+create table tbl_1(i1 int, j1 int);
+insert into tbl_1 values(1,2),(1,null), (null, 200), (45,68);
+create table tbl_2(i2 int, j2 int);
+insert into tbl_2 values(1,2),(1,null), (null, 200), (45,68);
+
+-- simple join
+explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2;
+select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2;
+
+explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2;
+select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2;
+
+explain cbo select * from tbl_1 full outer join tbl_2 on tbl_1.i1 = tbl_2.i2;
+select * from tbl_1 full outer join tbl_2 on tbl_1.i1 = tbl_2.i2;
+
+-- conjunction
+explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2;
+select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2;
+
+explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2;
+select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2;
+
+-- equi + non-equi
+explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2;
+select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2;
+
+explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2;
+select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2;
+
+explain cbo SELECT t0.col0, t0.col1
+FROM
+  (
+    SELECT i1 as col0, j1 as col1 FROM tbl_1
+  ) AS t0
+  LEFT JOIN
+  (
+    SELECT i2 as col0, j2 as col1 FROM tbl_2
+  ) AS t1
+ON t0.col0 = t1.col0 AND t0.col1 = t1.col1;
+
+SELECT t0.col0, t0.col1
+FROM
+  (
+    SELECT i1 as col0, j1 as col1 FROM tbl_1
+  ) AS t0
+  LEFT JOIN
+  (
+    SELECT i2 as col0, j2 as col1 FROM tbl_2
+  ) AS t1
+ON t0.col0 = t1.col0 AND t0.col1 = t1.col1;
+
+DROP TABLE tbl_1;
+DROP TABLE tbl_2;
diff --git a/ql/src/test/results/clientpositive/annotate_stats_join.q.out b/ql/src/test/results/clientpositive/annotate_stats_join.q.out
index 6c73d63..abe31e0 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_join.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_join.q.out
@@ -814,16 +814,20 @@ STAGE PLANS:
                 value expressions: _col2 (type: int)
           TableScan
             alias: dept_n1
+            filterExpr: (deptid is not null and deptname is not null) (type: boolean)
             Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: deptid (type: int), deptname (type: string)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: (deptid is not null and deptname is not null) (type: boolean)
               Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: string), _col0 (type: int)
-                sort order: ++
-                Map-reduce partition columns: _col1 (type: string), _col0 (type: int)
+              Select Operator
+                expressions: deptid (type: int), deptname (type: string)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: string), _col0 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col1 (type: string), _col0 (type: int)
+                  Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -947,17 +951,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: emp_n2
+            filterExpr: (deptid is not null and lastname is not null) (type: boolean)
             Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: lastname (type: string), deptid (type: int), locid (type: int)
-              outputColumnNames: _col0, _col1, _col2
+            Filter Operator
+              predicate: (deptid is not null and lastname is not null) (type: boolean)
               Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string), _col1 (type: int)
-                sort order: ++
-                Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
+              Select Operator
+                expressions: lastname (type: string), deptid (type: int), locid (type: int)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col2 (type: int)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
+                  Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col2 (type: int)
           TableScan
             alias: dept_n1
             Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/cbo_SortUnionTransposeRule.q.out b/ql/src/test/results/clientpositive/cbo_SortUnionTransposeRule.q.out
index 1beaa64..89beaa5 100644
--- a/ql/src/test/results/clientpositive/cbo_SortUnionTransposeRule.q.out
+++ b/ql/src/test/results/clientpositive/cbo_SortUnionTransposeRule.q.out
@@ -296,17 +296,21 @@ STAGE PLANS:
       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
-            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)
-                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)
+                  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: src1
             Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -402,17 +406,21 @@ STAGE PLANS:
       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
-            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)
-                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)
+                  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: src1
             Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -911,17 +919,21 @@ STAGE PLANS:
       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
-            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)
-                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)
+                  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
             Reduce Output Operator
               key expressions: _col0 (type: string)
@@ -1045,17 +1057,21 @@ STAGE PLANS:
       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
-            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)
-                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)
+                  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
             Reduce Output Operator
               key expressions: _col0 (type: string)
diff --git a/ql/src/test/results/clientpositive/cbo_rp_join0.q.out b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
index 76eaa52..8b072a5 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
@@ -735,17 +735,21 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 1615 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
             alias: cbo_t4:cbo_t1
+            filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), c_int (type: int)
-              outputColumnNames: key, c_int
-              Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: key (type: string)
-                sort order: +
-                Map-reduce partition columns: key (type: string)
-                Statistics: Num rows: 20 Data size: 1691 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: c_int (type: int)
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), c_int (type: int)
+                outputColumnNames: key, c_int
+                Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: key (type: string)
+                  sort order: +
+                  Map-reduce partition columns: key (type: string)
+                  Statistics: Num rows: 18 Data size: 1513 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: c_int (type: int)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -758,14 +762,14 @@ STAGE PLANS:
             2 key (type: string)
             3 key (type: string)
           outputColumnNames: key, c_int, key0, c_int0, key1, c_int2
-          Statistics: Num rows: 1080 Data size: 288093 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 972 Data size: 259257 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: key (type: string), c_int (type: int), key0 (type: string), c_int0 (type: int), key1 (type: string), c_int2 (type: int)
             outputColumnNames: key, c_int, p, q, x, b
-            Statistics: Num rows: 1080 Data size: 288093 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 972 Data size: 259257 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 1080 Data size: 288093 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 972 Data size: 259257 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/constant_prop_3.q.out b/ql/src/test/results/clientpositive/constant_prop_3.q.out
index c95a56a..293d8f8 100644
--- a/ql/src/test/results/clientpositive/constant_prop_3.q.out
+++ b/ql/src/test/results/clientpositive/constant_prop_3.q.out
@@ -355,10 +355,10 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: supplier_hive
-            filterExpr: (s_comment like '%Customer%Complaints%') (type: boolean)
+            filterExpr: ((s_comment like '%Customer%Complaints%') and s_suppkey is not null) (type: boolean)
             Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (s_comment like '%Customer%Complaints%') (type: boolean)
+              predicate: ((s_comment like '%Customer%Complaints%') and s_suppkey is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: s_suppkey (type: int)
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer8.q.out b/ql/src/test/results/clientpositive/correlationoptimizer8.q.out
index 69a6670..37a6c44 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer8.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer8.q.out
@@ -510,17 +510,21 @@ STAGE PLANS:
                 value expressions: _col1 (type: bigint)
           TableScan
             alias: x
+            filterExpr: key is not null (type: boolean)
             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
+            Filter Operator
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+              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
-                value expressions: _col1 (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -719,17 +723,21 @@ STAGE PLANS:
                     value expressions: _col1 (type: bigint)
           TableScan
             alias: x
+            filterExpr: key is not null (type: boolean)
             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
+            Filter Operator
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+              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
-                value expressions: _col1 (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Demux Operator
           Statistics: Num rows: 112 Data size: 12648 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/infer_join_preds.q.out b/ql/src/test/results/clientpositive/infer_join_preds.q.out
index 185d0f8..11dbfa1 100644
--- a/ql/src/test/results/clientpositive/infer_join_preds.q.out
+++ b/ql/src/test/results/clientpositive/infer_join_preds.q.out
@@ -263,17 +263,21 @@ STAGE PLANS:
                   value expressions: _col1 (type: string)
           TableScan
             alias: src1
+            filterExpr: key is not null (type: boolean)
             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
+            Filter Operator
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+              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
-                value expressions: _col1 (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -579,17 +583,21 @@ STAGE PLANS:
       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)
-                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)
+                  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: src1
             Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/innerjoin.q.out b/ql/src/test/results/clientpositive/innerjoin.q.out
index 709dbdb..9694ed1 100644
--- a/ql/src/test/results/clientpositive/innerjoin.q.out
+++ b/ql/src/test/results/clientpositive/innerjoin.q.out
@@ -1253,17 +1253,21 @@ STAGE PLANS:
                 value expressions: _col1 (type: string)
           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)
-                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)
+                  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:
diff --git a/ql/src/test/results/clientpositive/join45.q.out b/ql/src/test/results/clientpositive/join45.q.out
index fd639b9..826412d 100644
--- a/ql/src/test/results/clientpositive/join45.q.out
+++ b/ql/src/test/results/clientpositive/join45.q.out
@@ -766,7 +766,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 #### A masked pattern was here ####
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM src
@@ -811,17 +811,21 @@ STAGE PLANS:
                 value expressions: _col1 (type: string), _col2 (type: double)
           TableScan
             alias: b
+            filterExpr: key is not null (type: boolean)
             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
+            Filter Operator
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+              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
-                value expressions: _col1 (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -890,7 +894,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM src
 JOIN src1 a ON (a.key+src.key >= 100)
@@ -919,7 +923,7 @@ POSTHOOK: Input: default@src1
 98	val_98	273	val_273	98	val_98
 98	val_98	150	val_150	98	val_98
 98	val_98	401	val_401	98	val_98
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM src
@@ -964,17 +968,21 @@ STAGE PLANS:
                 value expressions: _col1 (type: string), _col2 (type: double)
           TableScan
             alias: a
+            filterExpr: key is not null (type: boolean)
             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
+            Filter Operator
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+              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
-                value expressions: _col1 (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -1043,7 +1051,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM src
 LEFT OUTER JOIN src1 a ON (a.key = src.key)
@@ -1072,7 +1080,7 @@ POSTHOOK: Input: default@src1
 35	val_35	NULL	NULL	66	val_66
 35	val_35	NULL	NULL	66	val_66
 35	val_35	NULL	NULL	66	val_66
-Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[7][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM src
@@ -1104,15 +1112,19 @@ STAGE PLANS:
       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), UDFToDouble(key) (type: double)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                sort order: 
+            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) (type: double)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double)
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double)
           TableScan
             alias: a
             Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1196,7 +1208,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[7][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM src
 JOIN src1 a ON (a.key+src.key >= 100)
@@ -1225,7 +1237,7 @@ NULL	NULL	NULL	NULL		val_409
 NULL	NULL	NULL	NULL		
 NULL	NULL	NULL	NULL		val_193
 NULL	NULL	NULL	NULL		val_265
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM src
@@ -1257,17 +1269,21 @@ STAGE PLANS:
       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), UDFToDouble(key) (type: double)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+            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) (type: double)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: string), _col2 (type: double)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string), _col2 (type: double)
           TableScan
             alias: a
             Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1349,7 +1365,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM src
 RIGHT OUTER JOIN src1 a ON (a.key = src.key)
@@ -1674,9 +1690,9 @@ POSTHOOK: Input: default@src1
 35	val_35	NULL	NULL	66	val_66
 35	val_35	NULL	NULL	66	val_66
 35	val_35	NULL	NULL	66	val_66
-Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-3:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-3: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
+Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM cbo_t1 t1
@@ -1796,7 +1812,7 @@ STAGE PLANS:
           TableScan
             Reduce Output Operator
               sort order: 
-              Statistics: Num rows: 100 Data size: 72921 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 90 Data size: 65601 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: int), _col9 (type: float), _col10 (type: boolean), _col11 (type: string)
       Reduce Operator Tree:
         Join Operator
@@ -1806,14 +1822,14 @@ STAGE PLANS:
             0 
             1 
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30
-          Statistics: Num rows: 800000 Data size: 1467199175 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 720000 Data size: 1320479175 Basic stats: COMPLETE Column stats: COMPLETE
           Filter Operator
             predicate: (((_col0 = _col25) or ((_col29 = _col17) and _col6)) and ((_col13 = _col25) or ((_col28 = _col16) and (_col14 = _col26))) and ((_col7 = _col25) or ((_col8 = _col26) and (_col27 = _col21)))) (type: boolean)
-            Statistics: Num rows: 337500 Data size: 618975000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 303750 Data size: 557077500 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: _col19 (type: string), _col20 (type: string), _col21 (type: int), _col22 (type: float), _col23 (type: boolean), _col24 (type: string), _col25 (type: string), _col26 (type: string), _col27 (type: int), _col28 (type: float), _col29 (type: boolean), _col30 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: int), _col10 (type: float), _col11 (type: boolean), _col12 (type: string), _col13 (type: string), _col14 (type: string), _col15 (type: int [...]
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29
-              Statistics: Num rows: 337500 Data size: 617625000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 303750 Data size: 555862500 Basic stats: COMPLETE Column stats: COMPLETE
               Limit
                 Number of rows: 10
                 Statistics: Num rows: 10 Data size: 18300 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1830,17 +1846,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: t1
+            filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 20 Data size: 7138 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-              Statistics: Num rows: 20 Data size: 7138 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 20 Data size: 7138 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 18 Data size: 6406 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 18 Data size: 6406 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 18 Data size: 6406 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
           TableScan
             alias: t2
             Statistics: Num rows: 20 Data size: 7138 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1862,7 +1882,7 @@ STAGE PLANS:
             0 _col0 (type: string)
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-          Statistics: Num rows: 100 Data size: 72921 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 90 Data size: 65601 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -1876,9 +1896,9 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-3:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-3: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
+Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM cbo_t1 t1
 RIGHT OUTER JOIN cbo_t1 t2 ON (t2.key = t1.key)
@@ -1902,8 +1922,8 @@ POSTHOOK: Input: default@cbo_t1
 POSTHOOK: Input: default@cbo_t1@dt=2014
 #### A masked pattern was here ####
 1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
-1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
 1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
+1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
 1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
 1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
 1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
diff --git a/ql/src/test/results/clientpositive/join46.q.out b/ql/src/test/results/clientpositive/join46.q.out
index 02cb625..3649968 100644
--- a/ql/src/test/results/clientpositive/join46.q.out
+++ b/ql/src/test/results/clientpositive/join46.q.out
@@ -79,17 +79,21 @@ STAGE PLANS:
                 value expressions: _col0 (type: int), _col2 (type: string)
           TableScan
             alias: test2_n0
+            filterExpr: value is not null (type: boolean)
             Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: int), value (type: int), col_2 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: int)
-                sort order: +
-                Map-reduce partition columns: _col1 (type: int)
-                Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: int), _col2 (type: string)
+            Filter Operator
+              predicate: value is not null (type: boolean)
+              Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int), value (type: int), col_2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col2 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -98,10 +102,10 @@ STAGE PLANS:
             0 _col1 (type: int)
             1 _col1 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -178,10 +182,10 @@ STAGE PLANS:
                 value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: boolean)
           TableScan
             alias: test2_n0
-            filterExpr: key BETWEEN 100 AND 102 (type: boolean)
+            filterExpr: (key BETWEEN 100 AND 102 and value is not null) (type: boolean)
             Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: key BETWEEN 100 AND 102 (type: boolean)
+              predicate: (key BETWEEN 100 AND 102 and value is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), value (type: int), col_2 (type: string)
@@ -379,17 +383,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: test1_n2
+            filterExpr: value is not null (type: boolean)
             Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: int), value (type: int), col_1 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: int)
-                sort order: +
-                Map-reduce partition columns: _col1 (type: int)
-                Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: int), _col2 (type: string)
+            Filter Operator
+              predicate: value is not null (type: boolean)
+              Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int), value (type: int), col_1 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col2 (type: string)
           TableScan
             alias: test2_n0
             Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
@@ -411,10 +419,10 @@ STAGE PLANS:
             0 _col1 (type: int)
             1 _col1 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-          Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -909,17 +917,21 @@ STAGE PLANS:
                 value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: boolean)
           TableScan
             alias: test2_n0
+            filterExpr: value is not null (type: boolean)
             Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: int), value (type: int), col_2 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: int)
-                sort order: +
-                Map-reduce partition columns: _col1 (type: int)
-                Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: int), _col2 (type: string)
+            Filter Operator
+              predicate: value is not null (type: boolean)
+              Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int), value (type: int), col_2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col2 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -929,14 +941,14 @@ STAGE PLANS:
             1 _col1 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
-          Statistics: Num rows: 11 Data size: 1947 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 9 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1420,17 +1432,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: test1_n2
+            filterExpr: value is not null (type: boolean)
             Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: int), value (type: int), col_1 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: int)
-                sort order: +
-                Map-reduce partition columns: _col1 (type: int)
-                Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: int), _col2 (type: string)
+            Filter Operator
+              predicate: value is not null (type: boolean)
+              Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int), value (type: int), col_1 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col2 (type: string)
           TableScan
             alias: test2_n0
             Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1453,14 +1469,14 @@ STAGE PLANS:
             1 _col1 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           residual filter predicates: {(_col0 BETWEEN 100 AND 102 or _col6)}
-          Statistics: Num rows: 10 Data size: 1850 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1910,7 +1926,7 @@ NULL	NULL	NULL	105	NULL	None
 101	2	Car	102	2	Del
 99	2	Mat	102	2	Del
 NULL	NULL	NULL	104	3	Fli
-Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM (
@@ -1968,19 +1984,6 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: test1_n2
-            Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: int), value (type: int), col_1 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: int)
-                sort order: +
-                Map-reduce partition columns: _col1 (type: int)
-                Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: int), _col2 (type: string)
-          TableScan
             alias: test2_n0
             Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
@@ -1993,20 +1996,37 @@ STAGE PLANS:
                 Map-reduce partition columns: _col1 (type: int)
                 Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
                 value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: boolean)
+          TableScan
+            alias: test1_n2
+            filterExpr: value is not null (type: boolean)
+            Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: value is not null (type: boolean)
+              Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int), value (type: int), col_1 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col2 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
-               Right Outer Join 0 to 1
+               Left Outer Join 0 to 1
           keys:
             0 _col1 (type: int)
             1 _col1 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-          residual filter predicates: {(_col0 BETWEEN 100 AND 102 or _col6)}
-          Statistics: Num rows: 10 Data size: 1850 Basic stats: COMPLETE Column stats: COMPLETE
+          residual filter predicates: {(_col4 BETWEEN 100 AND 102 or _col3)}
+          Statistics: Num rows: 8 Data size: 1560 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
+            expressions: _col4 (type: int), _col5 (type: int), _col6 (type: string), _col0 (type: int), _col1 (type: int), _col2 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -2020,12 +2040,12 @@ STAGE PLANS:
           TableScan
             Reduce Output Operator
               sort order: 
-              Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 1528 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
           TableScan
             Reduce Output Operator
               sort order: 
-              Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
       Reduce Operator Tree:
         Join Operator
@@ -2036,10 +2056,10 @@ STAGE PLANS:
             1 
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
           residual filter predicates: {(_col1 is null or (_col10 is null and (_col7 <> _col4)))}
-          Statistics: Num rows: 110 Data size: 41722 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 72 Data size: 27306 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 110 Data size: 41722 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 72 Data size: 27306 Basic stats: COMPLETE Column stats: COMPLETE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2063,17 +2083,21 @@ STAGE PLANS:
                 value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: boolean)
           TableScan
             alias: test2_n0
+            filterExpr: value is not null (type: boolean)
             Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: int), value (type: int), col_2 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: int)
-                sort order: +
-                Map-reduce partition columns: _col1 (type: int)
-                Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: int), _col2 (type: string)
+            Filter Operator
+              predicate: value is not null (type: boolean)
+              Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int), value (type: int), col_2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col2 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -2083,11 +2107,11 @@ STAGE PLANS:
             1 _col1 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
           residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
-          Statistics: Num rows: 11 Data size: 1947 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 9 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -2101,7 +2125,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM (
   SELECT test1_n2.key AS key1, test1_n2.value AS value1, test1_n2.col_1 AS col_1,
diff --git a/ql/src/test/results/clientpositive/join47.q.out b/ql/src/test/results/clientpositive/join47.q.out
index e9b6be4..02b4e7e 100644
--- a/ql/src/test/results/clientpositive/join47.q.out
+++ b/ql/src/test/results/clientpositive/join47.q.out
@@ -758,7 +758,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 #### A masked pattern was here ####
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM src
@@ -803,17 +803,21 @@ STAGE PLANS:
                 value expressions: _col1 (type: string), _col2 (type: double)
           TableScan
             alias: b
+            filterExpr: key is not null (type: boolean)
             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
+            Filter Operator
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+              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
-                value expressions: _col1 (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -880,7 +884,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM src
 JOIN src1 a ON (a.key+src.key >= 100)
@@ -909,7 +913,7 @@ POSTHOOK: Input: default@src1
 98	val_98	273	val_273	98	val_98
 98	val_98	150	val_150	98	val_98
 98	val_98	401	val_401	98	val_98
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM src
@@ -954,17 +958,21 @@ STAGE PLANS:
                 value expressions: _col1 (type: string), _col2 (type: double)
           TableScan
             alias: a
+            filterExpr: key is not null (type: boolean)
             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
+            Filter Operator
+              predicate: key is not null (type: boolean)
               Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+              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
-                value expressions: _col1 (type: string)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -1031,7 +1039,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM src
 LEFT OUTER JOIN src1 a ON (a.key = src.key)
@@ -1060,7 +1068,7 @@ POSTHOOK: Input: default@src1
 35	val_35	NULL	NULL	66	val_66
 35	val_35	NULL	NULL	66	val_66
 35	val_35	NULL	NULL	66	val_66
-Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[7][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM src
@@ -1092,15 +1100,19 @@ STAGE PLANS:
       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), UDFToDouble(key) (type: double)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                sort order: 
+            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) (type: double)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double)
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double)
           TableScan
             alias: a
             Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1182,7 +1194,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[6][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[7][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM src
 JOIN src1 a ON (a.key+src.key >= 100)
@@ -1211,7 +1223,7 @@ NULL	NULL	NULL	NULL		val_409
 NULL	NULL	NULL	NULL		
 NULL	NULL	NULL	NULL		val_193
 NULL	NULL	NULL	NULL		val_265
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM src
@@ -1243,17 +1255,21 @@ STAGE PLANS:
       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), UDFToDouble(key) (type: double)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+            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) (type: double)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: string), _col2 (type: double)
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string), _col2 (type: double)
           TableScan
             alias: a
             Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1333,7 +1349,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[11][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM src
 RIGHT OUTER JOIN src1 a ON (a.key = src.key)
@@ -1654,9 +1670,9 @@ POSTHOOK: Input: default@src1
 35	val_35	NULL	NULL	66	val_66
 35	val_35	NULL	NULL	66	val_66
 35	val_35	NULL	NULL	66	val_66
-Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-3:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-3: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
+Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM cbo_t1 t1
@@ -1776,7 +1792,7 @@ STAGE PLANS:
           TableScan
             Reduce Output Operator
               sort order: 
-              Statistics: Num rows: 100 Data size: 72921 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 90 Data size: 65601 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string), _col6 (type: string), _col7 (type: string), _col8 (type: int), _col9 (type: float), _col10 (type: boolean), _col11 (type: string)
       Reduce Operator Tree:
         Join Operator
@@ -1787,11 +1803,11 @@ STAGE PLANS:
             1 
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30
           residual filter predicates: {((_col0 = _col25) or ((_col29 = _col17) and _col6))} {((_col13 = _col25) or ((_col28 = _col16) and (_col14 = _col26)))} {((_col7 = _col25) or ((_col8 = _col26) and (_col27 = _col21)))}
-          Statistics: Num rows: 337500 Data size: 618975000 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 303750 Data size: 557077500 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col19 (type: string), _col20 (type: string), _col21 (type: int), _col22 (type: float), _col23 (type: boolean), _col24 (type: string), _col25 (type: string), _col26 (type: string), _col27 (type: int), _col28 (type: float), _col29 (type: boolean), _col30 (type: string), _col7 (type: string), _col8 (type: string), _col9 (type: int), _col10 (type: float), _col11 (type: boolean), _col12 (type: string), _col13 (type: string), _col14 (type: string), _col15 (type: int), [...]
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29
-            Statistics: Num rows: 337500 Data size: 617625000 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 303750 Data size: 555862500 Basic stats: COMPLETE Column stats: COMPLETE
             Limit
               Number of rows: 10
               Statistics: Num rows: 10 Data size: 18300 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1808,17 +1824,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: t1
+            filterExpr: key is not null (type: boolean)
             Statistics: Num rows: 20 Data size: 7138 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-              Statistics: Num rows: 20 Data size: 7138 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 20 Data size: 7138 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 18 Data size: 6406 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), value (type: string), c_int (type: int), c_float (type: float), c_boolean (type: boolean), dt (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 18 Data size: 6406 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 18 Data size: 6406 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: boolean), _col5 (type: string)
           TableScan
             alias: t2
             Statistics: Num rows: 20 Data size: 7138 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1840,7 +1860,7 @@ STAGE PLANS:
             0 _col0 (type: string)
             1 _col0 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
-          Statistics: Num rows: 100 Data size: 72921 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 90 Data size: 65601 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
             compressed: false
             table:
@@ -1854,9 +1874,9 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-3:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[19][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-2:MAPRED' is a cross product
-Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Stage-3: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
+Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: SELECT *
 FROM cbo_t1 t1
 RIGHT OUTER JOIN cbo_t1 t2 ON (t2.key = t1.key)
@@ -1880,8 +1900,8 @@ POSTHOOK: Input: default@cbo_t1
 POSTHOOK: Input: default@cbo_t1@dt=2014
 #### A masked pattern was here ####
 1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
-1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
 1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
+1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
 1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
 1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
 1	1	1	1.0	true	2014	1	1	1	1.0	true	2014	1	1	1	1.0	false	2014	1	1	1	1.0	true	2014	1	1	1	1.0	true	2014
diff --git a/ql/src/test/results/clientpositive/join_cond_pushdown_unqual5.q.out b/ql/src/test/results/clientpositive/join_cond_pushdown_unqual5.q.out
index a966d8c..ea25357 100644
--- a/ql/src/test/results/clientpositive/join_cond_pushdown_unqual5.q.out
+++ b/ql/src/test/results/clientpositive/join_cond_pushdown_unqual5.q.out
@@ -70,17 +70,21 @@ STAGE PLANS:
                 value expressions: _col0 (type: int), _col1 (type: int)
           TableScan
             alias: r
+            filterExpr: (rk1 is not null and rk2 is not null) (type: boolean)
             Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: ra (type: int), rk1 (type: string), rk2 (type: string)
-              outputColumnNames: _col0, _col1, _col2
+            Filter Operator
+              predicate: (rk1 is not null and rk2 is not null) (type: boolean)
               Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: string), _col2 (type: string)
-                sort order: ++
-                Map-reduce partition columns: _col1 (type: string), _col2 (type: string)
+              Select Operator
+                expressions: ra (type: int), rk1 (type: string), rk2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: int)
+                Reduce Output Operator
+                  key expressions: _col1 (type: string), _col2 (type: string)
+                  sort order: ++
+                  Map-reduce partition columns: _col1 (type: string), _col2 (type: string)
+                  Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -144,15 +148,19 @@ STAGE PLANS:
         $hdt$_1:r 
           TableScan
             alias: r
+            filterExpr: (rk1 is not null and rk2 is not null) (type: boolean)
             Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: ra (type: int), rk1 (type: string), rk2 (type: string)
-              outputColumnNames: _col0, _col1, _col2
+            Filter Operator
+              predicate: (rk1 is not null and rk2 is not null) (type: boolean)
               Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
-              HashTable Sink Operator
-                keys:
-                  0 _col2 (type: string), _col3 (type: string)
-                  1 _col1 (type: string), _col2 (type: string)
+              Select Operator
+                expressions: ra (type: int), rk1 (type: string), rk2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 2 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: string), _col3 (type: string)
+                    1 _col1 (type: string), _col2 (type: string)
 
   Stage: Stage-3
     Map Reduce
diff --git a/ql/src/test/results/clientpositive/join_emit_interval.q.out b/ql/src/test/results/clientpositive/join_emit_interval.q.out
index 9f3f01f..ef40660 100644
--- a/ql/src/test/results/clientpositive/join_emit_interval.q.out
+++ b/ql/src/test/results/clientpositive/join_emit_interval.q.out
@@ -79,17 +79,21 @@ STAGE PLANS:
                 value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: boolean)
           TableScan
             alias: test2_n4
+            filterExpr: value is not null (type: boolean)
             Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: int), value (type: int), col_2 (type: string)
-              outputColumnNames: _col0, _col1, _col2
-              Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col1 (type: int)
-                sort order: +
-                Map-reduce partition columns: _col1 (type: int)
-                Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col0 (type: int), _col2 (type: string)
+            Filter Operator
+              predicate: value is not null (type: boolean)
+              Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int), value (type: int), col_2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: int)
+                  Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col2 (type: string)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -101,14 +105,14 @@ STAGE PLANS:
             0 _col1 (type: int)
             1 _col1 (type: int)
           outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6
-          Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 9 Data size: 1521 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/join_filters_overlap.q.out b/ql/src/test/results/clientpositive/join_filters_overlap.q.out
index 41ebfcc..4eab694 100644
--- a/ql/src/test/results/clientpositive/join_filters_overlap.q.out
+++ b/ql/src/test/results/clientpositive/join_filters_overlap.q.out
@@ -23,10 +23,10 @@ FROM (SELECT `key`, `value`, `value` = 60 AS `=`, `value` = 50 AS `=3`
 FROM `default`.`a_n4`) AS `t`
 LEFT JOIN (SELECT `key`, CAST(50 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 50) AS `t1` ON `t`.`key` = `t1`.`key` AND `t`.`=3`
+WHERE `value` = 50 AND `key` IS NOT NULL) AS `t1` ON `t`.`key` = `t1`.`key` AND `t`.`=3`
 LEFT JOIN (SELECT `key`, CAST(60 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 60) AS `t3` ON `t`.`key` = `t3`.`key` AND `t`.`=`
+WHERE `value` = 60 AND `key` IS NOT NULL) AS `t3` ON `t`.`key` = `t3`.`key` AND `t`.`=`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-2 depends on stages: Stage-1
@@ -55,12 +55,12 @@ STAGE PLANS:
                 auto parallelism: false
           TableScan
             alias: b
-            filterExpr: (value = 50) (type: boolean)
+            filterExpr: ((value = 50) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 50) (type: boolean)
+              predicate: ((value = 50) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 50 (type: int)
@@ -178,12 +178,12 @@ STAGE PLANS:
               auto parallelism: false
           TableScan
             alias: c
-            filterExpr: (value = 60) (type: boolean)
+            filterExpr: ((value = 60) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 60) (type: boolean)
+              predicate: ((value = 60) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 60 (type: int)
@@ -355,12 +355,12 @@ POSTHOOK: Input: default@a_n4
 OPTIMIZED SQL: SELECT `t0`.`key`, `t0`.`value`, `t1`.`key` AS `key1`, `t1`.`value` AS `value1`, `t3`.`key` AS `key2`, `t3`.`value` AS `value2`
 FROM (SELECT `key`, CAST(50 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 50) AS `t0`
+WHERE `value` = 50 AND `key` IS NOT NULL) AS `t0`
 RIGHT JOIN (SELECT `key`, `value`, `value` = 60 AS `=`, `value` = 50 AS `=3`
 FROM `default`.`a_n4`) AS `t1` ON `t0`.`key` = `t1`.`key` AND `t1`.`=3`
 LEFT JOIN (SELECT `key`, CAST(60 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 60) AS `t3` ON `t1`.`key` = `t3`.`key` AND `t1`.`=`
+WHERE `value` = 60 AND `key` IS NOT NULL) AS `t3` ON `t1`.`key` = `t3`.`key` AND `t1`.`=`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-2 depends on stages: Stage-1
@@ -372,12 +372,12 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: a_n4
-            filterExpr: (value = 50) (type: boolean)
+            filterExpr: ((value = 50) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 50) (type: boolean)
+              predicate: ((value = 50) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 50 (type: int)
@@ -512,12 +512,12 @@ STAGE PLANS:
               auto parallelism: false
           TableScan
             alias: c
-            filterExpr: (value = 60) (type: boolean)
+            filterExpr: ((value = 60) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 60) (type: boolean)
+              predicate: ((value = 60) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 60 (type: int)
@@ -689,12 +689,12 @@ POSTHOOK: Input: default@a_n4
 OPTIMIZED SQL: SELECT `t0`.`key`, `t0`.`value`, `t1`.`key` AS `key1`, `t1`.`value` AS `value1`, `t3`.`key` AS `key2`, `t3`.`value` AS `value2`
 FROM (SELECT `key`, CAST(50 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 50) AS `t0`
+WHERE `value` = 50 AND `key` IS NOT NULL) AS `t0`
 RIGHT JOIN (SELECT `key`, `value`, `value` = 60 AS `=`, `value` = 50 AS `=3`
 FROM `default`.`a_n4`) AS `t1` ON `t0`.`key` = `t1`.`key` AND `t1`.`=3`
 LEFT JOIN (SELECT `key`, CAST(60 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 60) AS `t3` ON `t1`.`key` = `t3`.`key` AND `t1`.`=`
+WHERE `value` = 60 AND `key` IS NOT NULL) AS `t3` ON `t1`.`key` = `t3`.`key` AND `t1`.`=`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-2 depends on stages: Stage-1
@@ -706,12 +706,12 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: a_n4
-            filterExpr: (value = 50) (type: boolean)
+            filterExpr: ((value = 50) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 50) (type: boolean)
+              predicate: ((value = 50) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 50 (type: int)
@@ -846,12 +846,12 @@ STAGE PLANS:
               auto parallelism: false
           TableScan
             alias: c
-            filterExpr: (value = 60) (type: boolean)
+            filterExpr: ((value = 60) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 60) (type: boolean)
+              predicate: ((value = 60) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 60 (type: int)
@@ -1027,10 +1027,10 @@ FULL JOIN (SELECT `key`, `value`, `value` = 60 AS `=`, `value` = 50 AS `=3`
 FROM `default`.`a_n4`) AS `t0` ON `t`.`key` = `t0`.`key` AND `t`.`=3` AND `t0`.`=3`
 LEFT JOIN (SELECT `key`, CAST(60 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 60) AS `t2` ON `t0`.`key` = `t2`.`key` AND `t0`.`=`
+WHERE `value` = 60 AND `key` IS NOT NULL) AS `t2` ON `t0`.`key` = `t2`.`key` AND `t0`.`=`
 LEFT JOIN (SELECT `key`, CAST(40 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 40) AS `t4` ON `t`.`key` = `t4`.`key` AND `t`.`=`
+WHERE `value` = 40 AND `key` IS NOT NULL) AS `t4` ON `t`.`key` = `t4`.`key` AND `t`.`=`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-2 depends on stages: Stage-1
@@ -1179,12 +1179,12 @@ STAGE PLANS:
               auto parallelism: false
           TableScan
             alias: c
-            filterExpr: (value = 60) (type: boolean)
+            filterExpr: ((value = 60) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 60) (type: boolean)
+              predicate: ((value = 60) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 60 (type: int)
@@ -1325,12 +1325,12 @@ STAGE PLANS:
               auto parallelism: false
           TableScan
             alias: d
-            filterExpr: (value = 40) (type: boolean)
+            filterExpr: ((value = 40) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 40) (type: boolean)
+              predicate: ((value = 40) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 40 (type: int)
@@ -1495,13 +1495,13 @@ FROM (SELECT `key`, `value`, `value` = 40 AS `=`, `value` = 60 AS `=3`, `value`
 FROM `default`.`a_n4`) AS `t`
 LEFT JOIN (SELECT `key`, CAST(50 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 50) AS `t1` ON `t`.`key` = `t1`.`key` AND `t`.`=4`
+WHERE `value` = 50 AND `key` IS NOT NULL) AS `t1` ON `t`.`key` = `t1`.`key` AND `t`.`=4`
 LEFT JOIN (SELECT `key`, CAST(60 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 60) AS `t3` ON `t`.`key` = `t3`.`key` AND `t`.`=3`
+WHERE `value` = 60 AND `key` IS NOT NULL) AS `t3` ON `t`.`key` = `t3`.`key` AND `t`.`=3`
 LEFT JOIN (SELECT `key`, CAST(40 AS INTEGER) AS `value`
 FROM `default`.`a_n4`
-WHERE `value` = 40) AS `t5` ON `t`.`key` = `t5`.`key` AND `t`.`=`
+WHERE `value` = 40 AND `key` IS NOT NULL) AS `t5` ON `t`.`key` = `t5`.`key` AND `t`.`=`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-2 depends on stages: Stage-1
@@ -1531,12 +1531,12 @@ STAGE PLANS:
                 auto parallelism: false
           TableScan
             alias: b
-            filterExpr: (value = 50) (type: boolean)
+            filterExpr: ((value = 50) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 50) (type: boolean)
+              predicate: ((value = 50) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 50 (type: int)
@@ -1654,12 +1654,12 @@ STAGE PLANS:
               auto parallelism: false
           TableScan
             alias: c
-            filterExpr: (value = 60) (type: boolean)
+            filterExpr: ((value = 60) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 60) (type: boolean)
+              predicate: ((value = 60) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 60 (type: int)
@@ -1800,12 +1800,12 @@ STAGE PLANS:
               auto parallelism: false
           TableScan
             alias: d
-            filterExpr: (value = 40) (type: boolean)
+            filterExpr: ((value = 40) and key is not null) (type: boolean)
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (value = 40) (type: boolean)
+              predicate: ((value = 40) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), 40 (type: int)
diff --git a/ql/src/test/results/clientpositive/join_merging.q.out b/ql/src/test/results/clientpositive/join_merging.q.out
index 5b9c063..2688f83 100644
--- a/ql/src/test/results/clientpositive/join_merging.q.out
+++ b/ql/src/test/results/clientpositive/join_merging.q.out
@@ -23,23 +23,27 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: p2
+            filterExpr: p_partkey is not null (type: boolean)
             Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: p_partkey (type: int), p_size (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: p_partkey is not null (type: boolean)
               Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: int)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: int)
+              Select Operator
+                expressions: p_partkey (type: int), p_size (type: int)
+                outputColumnNames: _col0, _col1
                 Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: int)
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: int)
           TableScan
             alias: p1
-            filterExpr: (p_size > 10) (type: boolean)
+            filterExpr: ((p_size > 10) and p_partkey is not null) (type: boolean)
             Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (p_size > 10) (type: boolean)
+              predicate: ((p_size > 10) and p_partkey is not null) (type: boolean)
               Statistics: Num rows: 8 Data size: 64 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: p_partkey (type: int), p_size (type: int)
@@ -54,16 +58,16 @@ STAGE PLANS:
       Reduce Operator Tree:
         Join Operator
           condition map:
-               Right Outer Join 0 to 1
+               Inner Join 0 to 1
           keys:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1, _col3
-          Statistics: Num rows: 16 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col3 (type: int), _col0 (type: int), _col1 (type: int)
             outputColumnNames: _col0, _col1, _col2
-            Statistics: Num rows: 16 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
               table:
@@ -91,7 +95,7 @@ STAGE PLANS:
               key expressions: _col1 (type: int)
               sort order: +
               Map-reduce partition columns: _col1 (type: int)
-              Statistics: Num rows: 16 Data size: 136 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 8 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
               value expressions: _col0 (type: int), _col2 (type: int)
       Reduce Operator Tree:
         Join Operator
@@ -101,14 +105,14 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col1 (type: int)
           outputColumnNames: _col1, _col3
-          Statistics: Num rows: 42 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 34 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
             expressions: _col1 (type: int), _col3 (type: int)
             outputColumnNames: _col0, _col1
-            Statistics: Num rows: 42 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 34 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 42 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 34 Data size: 72 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/join_star.q.out b/ql/src/test/results/clientpositive/join_star.q.out
index 9caf12d..5702ff0 100644
--- a/ql/src/test/results/clientpositive/join_star.q.out
+++ b/ql/src/test/results/clientpositive/join_star.q.out
@@ -528,27 +528,35 @@ STAGE PLANS:
         $hdt$_1:dim1 
           TableScan
             alias: dim1
+            filterExpr: f1 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f1 (type: int), f2 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f1 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col2 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f1 (type: int), f2 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: int)
+                    1 _col0 (type: int)
         $hdt$_2:dim2 
           TableScan
             alias: dim2
+            filterExpr: f3 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f3 (type: int), f4 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f3 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col4 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f3 (type: int), f4 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col4 (type: int)
+                    1 _col0 (type: int)
 
   Stage: Stage-5
     Map Reduce
@@ -691,87 +699,115 @@ STAGE PLANS:
         $hdt$_1:dim1 
           TableScan
             alias: dim1
+            filterExpr: f1 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f1 (type: int), f2 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f1 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col2 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f1 (type: int), f2 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col2 (type: int)
+                    1 _col0 (type: int)
         $hdt$_2:dim2 
           TableScan
             alias: dim2
+            filterExpr: f3 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f3 (type: int), f4 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f3 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col5 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f3 (type: int), f4 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col5 (type: int)
+                    1 _col0 (type: int)
         $hdt$_3:dim3 
           TableScan
             alias: dim3
+            filterExpr: f5 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f5 (type: int), f6 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f5 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col3 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f5 (type: int), f6 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col3 (type: int)
+                    1 _col0 (type: int)
         $hdt$_4:dim4 
           TableScan
             alias: dim4
+            filterExpr: f7 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f7 (type: int), f8 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f7 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col9 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f7 (type: int), f8 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col9 (type: int)
+                    1 _col0 (type: int)
         $hdt$_5:dim5 
           TableScan
             alias: dim5
+            filterExpr: f9 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f9 (type: int), f10 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f9 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col11 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f9 (type: int), f10 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col11 (type: int)
+                    1 _col0 (type: int)
         $hdt$_6:dim6 
           TableScan
             alias: dim6
+            filterExpr: f11 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f11 (type: int), f12 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f11 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col9 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f11 (type: int), f12 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col9 (type: int)
+                    1 _col0 (type: int)
         $hdt$_7:dim7 
           TableScan
             alias: dim7
+            filterExpr: f13 is not null (type: boolean)
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: f13 (type: int), f14 (type: int)
-              outputColumnNames: _col0, _col1
+            Filter Operator
+              predicate: f13 is not null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-              HashTable Sink Operator
-                keys:
-                  0 _col15 (type: int)
-                  1 _col0 (type: int)
+              Select Operator
+                expressions: f13 (type: int), f14 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                HashTable Sink Operator
+                  keys:
+                    0 _col15 (type: int)
+                    1 _col0 (type: int)
 
   Stage: Stage-15
     Map Reduce
diff --git a/ql/src/test/results/clientpositive/lineage1.q.out b/ql/src/test/results/clientpositive/lineage1.q.out
index 4a2ca45..e63bb24 100644
--- a/ql/src/test/results/clientpositive/lineage1.q.out
+++ b/ql/src/test/results/clientpositive/lineage1.q.out
@@ -56,17 +56,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: p1
+            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)
-                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)
+                  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: t1
             Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
@@ -236,17 +240,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: p2
+            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)
-                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)
+                  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: t2
             Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_14.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_14.q.out
index cf7252f..0799a41 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_14.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_14.q.out
@@ -62,11 +62,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -164,11 +168,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: b
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_15.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_15.q.out
index b6e0ebf..c2809d3 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_15.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_15.q.out
@@ -62,11 +62,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -143,11 +147,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: b
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_16.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_16.q.out
index 4e15692..6842815 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_16.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_16.q.out
@@ -222,18 +222,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: bucket_small_n17
-                  filterExpr: pri BETWEEN 1L AND 2L (type: boolean)
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 236 Data size: 45636 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: bigint), value (type: string)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 236 Data size: 45636 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: bigint)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: bigint)
-                      Statistics: Num rows: 236 Data size: 45636 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 225 Data size: 43508 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: bigint), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 225 Data size: 43508 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: bigint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: bigint)
+                        Statistics: Num rows: 225 Data size: 43508 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -246,14 +249,14 @@ STAGE PLANS:
                   0 _col0 (type: bigint)
                   1 _col0 (type: bigint)
                 outputColumnNames: _col0, _col1, _col3
-                Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: bigint), _col1 (type: string), _col3 (type: string), 'day1' (type: string), 1 (type: int)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                  Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -370,12 +373,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: bucket_small_n17
-                  filterExpr: pri BETWEEN 1L AND 2L (type: boolean)
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 236 Data size: 45636 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: bigint), value (type: string)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 236 Data size: 45636 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 225 Data size: 43508 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: bigint), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 225 Data size: 43508 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: bucket_big_n17
@@ -392,14 +398,14 @@ STAGE PLANS:
                         0 _col0 (type: bigint)
                         1 _col0 (type: bigint)
                       outputColumnNames: _col0, _col1, _col3
-                      Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
                         expressions: _col0 (type: bigint), _col1 (type: string), _col3 (type: string), 'day1' (type: string), 1 (type: int)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                        Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -626,12 +632,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: bucket_small_n17
-                  filterExpr: pri BETWEEN 1L AND 2L (type: boolean)
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 236 Data size: 45636 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: bigint), value (type: string)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 236 Data size: 45636 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 225 Data size: 43508 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: bigint), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 225 Data size: 43508 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: bucket_big_n17
@@ -648,14 +657,14 @@ STAGE PLANS:
                         0 _col0 (type: bigint)
                         1 _col0 (type: bigint)
                       outputColumnNames: _col0, _col1, _col3
-                      Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                       Select Operator
                         expressions: _col0 (type: bigint), _col1 (type: string), _col3 (type: string), 'day1' (type: string), 1 (type: int)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                        Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 259 Data size: 50199 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 247 Data size: 47858 Basic stats: COMPLETE Column stats: NONE
                           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/check_constraint.q.out b/ql/src/test/results/clientpositive/llap/check_constraint.q.out
index 7b794ba..297d892 100644
--- a/ql/src/test/results/clientpositive/llap/check_constraint.q.out
+++ b/ql/src/test/results/clientpositive/llap/check_constraint.q.out
@@ -1563,17 +1563,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s2
+                  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), CAST( key AS decimal(5,2)) (type: decimal(5,2))
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 500 Data size: 99500 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  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), CAST( key AS decimal(5,2)) (type: decimal(5,2))
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 99500 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: decimal(5,2))
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 99500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: decimal(5,2))
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
diff --git a/ql/src/test/results/clientpositive/llap/constprog_dpp.q.out b/ql/src/test/results/clientpositive/llap/constprog_dpp.q.out
index eef365b..5653d40 100644
--- a/ql/src/test/results/clientpositive/llap/constprog_dpp.q.out
+++ b/ql/src/test/results/clientpositive/llap/constprog_dpp.q.out
@@ -57,37 +57,39 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 2 llap
-      File Output Operator [FS_17]
-        Merge Join Operator [MERGEJOIN_21] (rows=1 width=4)
-          Conds:RS_33._col0=RS_36._col0(Left Outer),Output:["_col0"]
+      File Output Operator [FS_19]
+        Merge Join Operator [MERGEJOIN_23] (rows=1 width=4)
+          Conds:RS_35._col0=RS_39._col0(Left Outer),Output:["_col0"]
         <-Map 1 [SIMPLE_EDGE] vectorized, llap
-          SHUFFLE [RS_33]
+          SHUFFLE [RS_35]
             PartitionCols:_col0
-            Select Operator [SEL_32] (rows=1 width=4)
+            Select Operator [SEL_34] (rows=1 width=4)
               Output:["_col0"]
               TableScan [TS_0] (rows=1 width=4)
                 default@tb1,a,Tbl:COMPLETE,Col:NONE,Output:["id"]
         <-Reducer 5 [SIMPLE_EDGE] vectorized, llap
-          SHUFFLE [RS_36]
+          SHUFFLE [RS_39]
             PartitionCols:_col0
-            Limit [LIM_35] (rows=1 width=2)
-              Number of rows:1
-              Select Operator [SEL_34] (rows=1 width=2)
-                Output:["_col0"]
-              <-Union 4 [CUSTOM_SIMPLE_EDGE]
-                <-Map 3 [CONTAINS] vectorized, llap
-                  Reduce Output Operator [RS_39]
-                    Limit [LIM_38] (rows=1 width=2)
-                      Number of rows:1
-                      Select Operator [SEL_37] (rows=1 width=4)
-                        Output:["_col0"]
-                        TableScan [TS_22] (rows=1 width=4)
-                          Output:["id"]
-                <-Map 6 [CONTAINS] vectorized, llap
-                  Reduce Output Operator [RS_42]
-                    Limit [LIM_41] (rows=1 width=2)
-                      Number of rows:1
-                      Select Operator [SEL_40] (rows=1 width=0)
-                        Output:["_col0"]
-                        TableScan [TS_27] (rows=1 width=0)
+            Filter Operator [FIL_38] (rows=1 width=2)
+              predicate:_col0 is not null
+              Limit [LIM_37] (rows=1 width=2)
+                Number of rows:1
+                Select Operator [SEL_36] (rows=1 width=2)
+                  Output:["_col0"]
+                <-Union 4 [CUSTOM_SIMPLE_EDGE]
+                  <-Map 3 [CONTAINS] vectorized, llap
+                    Reduce Output Operator [RS_42]
+                      Limit [LIM_41] (rows=1 width=2)
+                        Number of rows:1
+                        Select Operator [SEL_40] (rows=1 width=4)
+                          Output:["_col0"]
+                          TableScan [TS_24] (rows=1 width=4)
+                            Output:["id"]
+                  <-Map 6 [CONTAINS] vectorized, llap
+                    Reduce Output Operator [RS_45]
+                      Limit [LIM_44] (rows=1 width=2)
+                        Number of rows:1
+                        Select Operator [SEL_43] (rows=1 width=0)
+                          Output:["_col0"]
+                          TableScan [TS_29] (rows=1 width=0)
 
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out
index 4ba041d..f21e33a 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer1.q.out
@@ -846,16 +846,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  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)
-                      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)
+                        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
         Reducer 2 
@@ -996,16 +1000,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  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)
-                      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)
+                        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
         Reducer 2 
@@ -1146,16 +1154,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  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)
-                      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)
+                        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
         Reducer 2 
@@ -1296,16 +1308,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  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)
-                      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)
+                        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
         Reducer 2 
@@ -1443,16 +1459,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: (key is not null and value 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 and value 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), _col1 (type: string)
-                      sort order: ++
-                      Map-reduce partition columns: _col0 (type: string), _col1 (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
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1581,16 +1601,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: (key is not null and value 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 and value 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), _col1 (type: string)
-                      sort order: ++
-                      Map-reduce partition columns: _col0 (type: string), _col1 (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
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1706,16 +1730,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: x
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 25 Data size: 2150 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: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -1856,16 +1884,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: x
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 25 Data size: 2150 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: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -2006,16 +2038,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: x
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 25 Data size: 2150 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: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
@@ -2156,16 +2192,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: x
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 25 Data size: 2150 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: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out
index c0909fe..2f6282e 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer2.q.out
@@ -379,20 +379,24 @@ 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
-                  Group By Operator
-                    aggregations: count(value)
-                    keys: key (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: bigint)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -535,20 +539,24 @@ 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
-                  Group By Operator
-                    aggregations: count(value)
-                    keys: key (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 25 Data size: 4375 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: bigint)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 12 Data size: 1128 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -667,20 +675,24 @@ 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
-                  Group By Operator
-                    aggregations: count(value)
-                    keys: key (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: bigint)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -823,20 +835,24 @@ 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
-                  Group By Operator
-                    aggregations: count(value)
-                    keys: key (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: bigint)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
index 66e5bff..45a646c 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
@@ -790,16 +790,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -815,16 +819,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: z
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1027,16 +1035,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1052,16 +1064,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: z
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1264,16 +1280,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1289,16 +1309,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: z
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1500,16 +1524,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1525,16 +1553,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: x
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1736,16 +1768,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1761,16 +1797,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: x
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1972,16 +2012,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
@@ -1997,16 +2041,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: x
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Map Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
index 87450c5..6f1f20f 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
@@ -3145,6 +3145,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcpart
+                  filterExpr: ds is not null (type: boolean)
                   Statistics: Num rows: 2000 Data size: 389248 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: ds (type: string)
@@ -3234,6 +3235,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcpart
+                  filterExpr: ds is not null (type: boolean)
                   Statistics: Num rows: 2000 Data size: 389248 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: ds (type: string)
@@ -5856,6 +5858,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcpart
+                  filterExpr: ds is not null (type: boolean)
                   Statistics: Num rows: 2000 Data size: 389248 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: ds (type: string)
@@ -5925,6 +5928,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcpart
+                  filterExpr: ds is not null (type: boolean)
                   Statistics: Num rows: 2000 Data size: 389248 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: ds (type: string)
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index 7c45ac2..8d50790 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -1125,25 +1125,27 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 2 llap
-      File Output Operator [FS_8]
-        Select Operator [SEL_7] (rows=85 width=7)
+      File Output Operator [FS_9]
+        Select Operator [SEL_8] (rows=77 width=7)
           Output:["_col0","_col1"]
-          Merge Join Operator [MERGEJOIN_16] (rows=85 width=7)
-            Conds:RS_4._col0=RS_5._col0(Left Outer),Output:["_col1","_col3"]
+          Merge Join Operator [MERGEJOIN_17] (rows=77 width=7)
+            Conds:RS_5._col0=RS_6._col0(Left Outer),Output:["_col1","_col3"]
           <-Map 1 [SIMPLE_EDGE] llap
-            SHUFFLE [RS_4]
+            SHUFFLE [RS_5]
               PartitionCols:_col0
               Select Operator [SEL_1] (rows=20 width=84)
                 Output:["_col0","_col1"]
                 TableScan [TS_0] (rows=20 width=84)
                   default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int"]
           <-Map 3 [SIMPLE_EDGE] llap
-            SHUFFLE [RS_5]
+            SHUFFLE [RS_6]
               PartitionCols:_col0
-              Select Operator [SEL_3] (rows=20 width=84)
+              Select Operator [SEL_4] (rows=18 width=84)
                 Output:["_col0","_col1"]
-                TableScan [TS_2] (rows=20 width=84)
-                  default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int"]
+                Filter Operator [FIL_11] (rows=18 width=84)
+                  predicate:key is not null
+                  TableScan [TS_2] (rows=20 width=84)
+                    default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int"]
 
 PREHOOK: query: explain select cbo_t1.c_int, cbo_t2.c_int from cbo_t1 full outer join  cbo_t2 on cbo_t1.key=cbo_t2.key
 PREHOOK: type: QUERY
@@ -2786,24 +2788,24 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 4 llap
-      File Output Operator [FS_32]
-        Select Operator [SEL_31] (rows=27 width=125)
+      File Output Operator [FS_34]
+        Select Operator [SEL_33] (rows=27 width=125)
           Output:["_col0","_col1"]
         <-Reducer 3 [SIMPLE_EDGE] llap
-          SHUFFLE [RS_30]
-            Select Operator [SEL_29] (rows=27 width=125)
+          SHUFFLE [RS_32]
+            Select Operator [SEL_31] (rows=27 width=125)
               Output:["_col0","_col1"]
-              Filter Operator [FIL_28] (rows=27 width=141)
+              Filter Operator [FIL_30] (rows=27 width=141)
                 predicate:(((_col3 >= _col2) or (_col2 = 0L) or _col5 is not null or _col1 is null) and (_col1 is not null or (_col2 = 0L) or _col5 is not null) and (_col5 is null or (_col2 = 0L)))
-                Select Operator [SEL_27] (rows=27 width=141)
+                Select Operator [SEL_29] (rows=27 width=141)
                   Output:["_col0","_col1","_col2","_col3","_col5"]
-                  Merge Join Operator [MERGEJOIN_38] (rows=27 width=141)
+                  Merge Join Operator [MERGEJOIN_40] (rows=27 width=141)
                     Conds:(Inner),Output:["_col0","_col1","_col3","_col4","_col5"]
                   <-Reducer 6 [CUSTOM_SIMPLE_EDGE] llap
-                    PARTITION_ONLY_SHUFFLE [RS_25]
-                      Group By Operator [GBY_19] (rows=1 width=16)
+                    PARTITION_ONLY_SHUFFLE [RS_27]
+                      Group By Operator [GBY_21] (rows=1 width=16)
                         Output:["_col0","_col1"],aggregations:["count()","count(_col0)"]
-                        Select Operator [SEL_15] (rows=1 width=16)
+                        Select Operator [SEL_17] (rows=1 width=16)
                           Output:["_col0"]
                           Group By Operator [GBY_7] (rows=1 width=16)
                             Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"]
@@ -2811,22 +2813,24 @@ Stage-0
                             PARTITION_ONLY_SHUFFLE [RS_6]
                               Group By Operator [GBY_5] (rows=1 width=16)
                                 Output:["_col0","_col1"],aggregations:["sum(p_size)","count(p_size)"]
-                                Filter Operator [FIL_35] (rows=8 width=4)
+                                Filter Operator [FIL_37] (rows=8 width=4)
                                   predicate:(p_size < 10)
                                   TableScan [TS_2] (rows=26 width=4)
                                     default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_size"]
                   <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap
-                    PARTITION_ONLY_SHUFFLE [RS_24]
-                      Merge Join Operator [MERGEJOIN_37] (rows=27 width=125)
-                        Conds:RS_21.UDFToDouble(_col1)=RS_22._col0(Left Outer),Output:["_col0","_col1","_col3"]
+                    PARTITION_ONLY_SHUFFLE [RS_26]
+                      Merge Join Operator [MERGEJOIN_39] (rows=27 width=125)
+                        Conds:RS_23.UDFToDouble(_col1)=RS_24._col0(Left Outer),Output:["_col0","_col1","_col3"]
                       <-Reducer 6 [SIMPLE_EDGE] llap
-                        PARTITION_ONLY_SHUFFLE [RS_22]
+                        PARTITION_ONLY_SHUFFLE [RS_24]
                           PartitionCols:_col0
-                          Select Operator [SEL_8] (rows=1 width=12)
+                          Select Operator [SEL_10] (rows=1 width=12)
                             Output:["_col0","_col1"]
-                             Please refer to the previous Group By Operator [GBY_7]
+                            Filter Operator [FIL_9] (rows=1 width=16)
+                              predicate:(UDFToDouble(_col0) is not null and _col1 is not null)
+                               Please refer to the previous Group By Operator [GBY_7]
                       <-Map 1 [SIMPLE_EDGE] llap
-                        SHUFFLE [RS_21]
+                        SHUFFLE [RS_23]
                           PartitionCols:UDFToDouble(_col1)
                           Select Operator [SEL_1] (rows=26 width=125)
                             Output:["_col0","_col1"]
@@ -6497,27 +6501,29 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 3 llap
-      File Output Operator [FS_14]
-        Group By Operator [GBY_12] (rows=1 width=24)
+      File Output Operator [FS_15]
+        Group By Operator [GBY_13] (rows=1 width=24)
           Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
         <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap
-          PARTITION_ONLY_SHUFFLE [RS_11]
-            Group By Operator [GBY_10] (rows=1 width=24)
+          PARTITION_ONLY_SHUFFLE [RS_12]
+            Group By Operator [GBY_11] (rows=1 width=24)
               Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
             <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
-              PARTITION_ONLY_SHUFFLE [RS_9]
+              PARTITION_ONLY_SHUFFLE [RS_10]
                 PartitionCols:rand()
-                Select Operator [SEL_7] (rows=1 width=404)
+                Select Operator [SEL_8] (rows=1 width=404)
                   Output:["_col0","_col1","_col2"]
-                  Map Join Operator [MAPJOIN_22] (rows=1 width=404)
-                    Conds:SEL_1._col2=RS_5._col1(Left Outer),Output:["_col0","_col1","_col3"]
+                  Map Join Operator [MAPJOIN_23] (rows=1 width=404)
+                    Conds:SEL_1._col2=RS_6._col1(Left Outer),Output:["_col0","_col1","_col3"]
                   <-Map 4 [BROADCAST_EDGE] llap
-                    BROADCAST [RS_5]
+                    BROADCAST [RS_6]
                       PartitionCols:_col1
-                      Select Operator [SEL_3] (rows=1 width=184)
+                      Select Operator [SEL_4] (rows=1 width=184)
                         Output:["_col0","_col1"]
-                        TableScan [TS_2] (rows=1 width=184)
-                          default@t2_n70,c,Tbl:COMPLETE,Col:NONE,Output:["key"]
+                        Filter Operator [FIL_17] (rows=1 width=184)
+                          predicate:UDFToDouble(key) is not null
+                          TableScan [TS_2] (rows=1 width=184)
+                            default@t2_n70,c,Tbl:COMPLETE,Col:NONE,Output:["key"]
                   <-Select Operator [SEL_1] (rows=1 width=368)
                       Output:["_col0","_col1","_col2"]
                       TableScan [TS_0] (rows=1 width=368)
@@ -6545,30 +6551,32 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 4 llap
-      File Output Operator [FS_14]
-        Group By Operator [GBY_12] (rows=1 width=24)
+      File Output Operator [FS_15]
+        Group By Operator [GBY_13] (rows=1 width=24)
           Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
         <-Reducer 3 [CUSTOM_SIMPLE_EDGE] llap
-          PARTITION_ONLY_SHUFFLE [RS_11]
-            Group By Operator [GBY_10] (rows=1 width=24)
+          PARTITION_ONLY_SHUFFLE [RS_12]
+            Group By Operator [GBY_11] (rows=1 width=24)
               Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)"]
             <-Map 2 [CUSTOM_SIMPLE_EDGE] llap
-              PARTITION_ONLY_SHUFFLE [RS_9]
+              PARTITION_ONLY_SHUFFLE [RS_10]
                 PartitionCols:rand()
-                Select Operator [SEL_7] (rows=1 width=404)
+                Select Operator [SEL_8] (rows=1 width=404)
                   Output:["_col0","_col1","_col2"]
-                  Map Join Operator [MAPJOIN_17] (rows=1 width=404)
-                    Conds:RS_4._col2=SEL_3._col1(Right Outer),Output:["_col0","_col1","_col3"]
+                  Map Join Operator [MAPJOIN_18] (rows=1 width=404)
+                    Conds:RS_5._col2=SEL_4._col1(Right Outer),Output:["_col0","_col1","_col3"]
                   <-Map 1 [BROADCAST_EDGE] llap
-                    BROADCAST [RS_4]
+                    BROADCAST [RS_5]
                       PartitionCols:_col2
-                      Select Operator [SEL_1] (rows=1 width=368)
+                      Select Operator [SEL_2] (rows=1 width=368)
                         Output:["_col0","_col1","_col2"]
-                        TableScan [TS_0] (rows=1 width=368)
-                          default@t1_n119,a,Tbl:COMPLETE,Col:NONE,Output:["key","val"]
-                  <-Select Operator [SEL_3] (rows=1 width=184)
+                        Filter Operator [FIL_17] (rows=1 width=368)
+                          predicate:UDFToDouble(key) is not null
+                          TableScan [TS_0] (rows=1 width=368)
+                            default@t1_n119,a,Tbl:COMPLETE,Col:NONE,Output:["key","val"]
+                  <-Select Operator [SEL_4] (rows=1 width=184)
                       Output:["_col0","_col1"]
-                      TableScan [TS_2] (rows=1 width=184)
+                      TableScan [TS_3] (rows=1 width=184)
                         default@t2_n70,c,Tbl:COMPLETE,Col:NONE,Output:["key"]
 
 PREHOOK: query: explain FROM T1_n119 a FULL OUTER JOIN T2_n70 c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key))
@@ -6642,27 +6650,29 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 3 llap
-      File Output Operator [FS_14]
-        Group By Operator [GBY_12] (rows=1 width=16)
+      File Output Operator [FS_15]
+        Group By Operator [GBY_13] (rows=1 width=16)
           Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
         <-Reducer 2 [CUSTOM_SIMPLE_EDGE] llap
-          PARTITION_ONLY_SHUFFLE [RS_11]
-            Group By Operator [GBY_10] (rows=1 width=16)
+          PARTITION_ONLY_SHUFFLE [RS_12]
+            Group By Operator [GBY_11] (rows=1 width=16)
               Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"]
             <-Map 1 [CUSTOM_SIMPLE_EDGE] llap
-              PARTITION_ONLY_SHUFFLE [RS_9]
+              PARTITION_ONLY_SHUFFLE [RS_10]
                 PartitionCols:rand()
-                Select Operator [SEL_7] (rows=1 width=202)
+                Select Operator [SEL_8] (rows=1 width=202)
                   Output:["_col0","_col1"]
-                  Map Join Operator [MAPJOIN_17] (rows=1 width=202)
-                    Conds:SEL_1._col1=RS_5._col1(Left Outer),Output:["_col0","_col2"]
+                  Map Join Operator [MAPJOIN_18] (rows=1 width=202)
+                    Conds:SEL_1._col1=RS_6._col1(Left Outer),Output:["_col0","_col2"]
                   <-Map 4 [BROADCAST_EDGE] llap
-                    BROADCAST [RS_5]
+                    BROADCAST [RS_6]
                       PartitionCols:_col1
-                      Select Operator [SEL_3] (rows=1 width=368)
+                      Select Operator [SEL_4] (rows=1 width=368)
                         Output:["_col0","_col1"]
-                        TableScan [TS_2] (rows=1 width=368)
-                          default@t1_n119,v,Tbl:COMPLETE,Col:NONE,Output:["key","val"]
+                        Filter Operator [FIL_17] (rows=1 width=368)
+                          predicate:UDFToDouble(key) is not null
+                          TableScan [TS_2] (rows=1 width=368)
+                            default@t1_n119,v,Tbl:COMPLETE,Col:NONE,Output:["key","val"]
                   <-Select Operator [SEL_1] (rows=1 width=184)
                       Output:["_col0","_col1"]
                       TableScan [TS_0] (rows=1 width=184)
diff --git a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
index 63c455b..5b2289d 100644
--- a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
@@ -559,7 +559,7 @@ STAGE PLANS:
                         1 _col0 (type: int)
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 18466 Data size: 147728 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         minReductionHashAggr: 0.0
@@ -576,16 +576,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: cd
+                  filterExpr: cint is not null (type: boolean)
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: cint (type: int)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
-                      Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cint is not null (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -675,7 +679,7 @@ STAGE PLANS:
                         1 _col0 (type: int)
                       input vertices:
                         1 Map 3
-                      Statistics: Num rows: 24737 Data size: 197896 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 18466 Data size: 147728 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         minReductionHashAggr: 0.0
@@ -692,16 +696,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: cd
+                  filterExpr: cint is not null (type: boolean)
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: cint (type: int)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
-                      Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cint is not null (type: boolean)
+                    Statistics: Num rows: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 9173 Data size: 27396 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
diff --git a/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out b/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out
index 338f840..067e42b 100644
--- a/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out
+++ b/ql/src/test/results/clientpositive/llap/join32_lessSize.q.out
@@ -1769,7 +1769,8 @@ INNER JOIN ((SELECT `key`, `value`
 FROM `default`.`src1`
 WHERE `value` IS NOT NULL) AS `t2`
 LEFT JOIN (SELECT `key`
-FROM `default`.`src`) AS `t3` ON `t2`.`key` = `t3`.`key`) ON `t0`.`value` = `t2`.`value`
+FROM `default`.`src`
+WHERE `key` IS NOT NULL) AS `t4` ON `t2`.`key` = `t4`.`key`) ON `t0`.`value` = `t2`.`value`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-2 depends on stages: Stage-1
@@ -1869,20 +1870,25 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: y
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
-                  Select Operator
-                    expressions: key (type: string)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    isSamplingPred: false
+                    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: a
-                      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
-                      tag: 1
-                      auto parallelism: true
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: a
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                        tag: 1
+                        auto parallelism: true
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Path -> Alias:
diff --git a/ql/src/test/results/clientpositive/llap/join46.q.out b/ql/src/test/results/clientpositive/llap/join46.q.out
index ec58429..0267f2d 100644
--- a/ql/src/test/results/clientpositive/llap/join46.q.out
+++ b/ql/src/test/results/clientpositive/llap/join46.q.out
@@ -89,17 +89,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n0
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -112,10 +116,10 @@ STAGE PLANS:
                   0 _col1 (type: int)
                   1 _col1 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -202,10 +206,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n0
-                  filterExpr: key BETWEEN 100 AND 102 (type: boolean)
+                  filterExpr: (key BETWEEN 100 AND 102 and value is not null) (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: key BETWEEN 100 AND 102 (type: boolean)
+                    predicate: (key BETWEEN 100 AND 102 and value is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int), value (type: int), col_2 (type: string)
@@ -427,17 +431,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test1_n2
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_1 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -467,10 +475,10 @@ STAGE PLANS:
                   0 _col1 (type: int)
                   1 _col1 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 1237 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7 Data size: 1237 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1031,17 +1039,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n0
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1055,14 +1067,14 @@ STAGE PLANS:
                   1 _col1 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
-                Statistics: Num rows: 11 Data size: 1947 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1608,17 +1620,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test1_n2
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_1 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -1649,14 +1665,14 @@ STAGE PLANS:
                   1 _col1 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 residual filter predicates: {(_col0 BETWEEN 100 AND 102 or _col6)}
-                Statistics: Num rows: 10 Data size: 1850 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 7 Data size: 1265 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7 Data size: 1237 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 1237 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2162,7 +2178,7 @@ NULL	NULL	None	NULL	NULL	NULL
 101	2	Car	102	2	Del
 101	2	Car	103	2	Ema
 NULL	NULL	NULL	104	3	Fli
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM (
@@ -2226,55 +2242,61 @@ STAGE PLANS:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: test1_n2
-                  Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  alias: test2_n0
+                  Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string), key BETWEEN 100 AND 102 (type: boolean)
+                    expressions: key (type: int), value (type: int), col_2 (type: string), key BETWEEN 100 AND 102 (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 6 Data size: 596 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col1 (type: int)
                       sort order: +
                       Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 6 Data size: 596 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: boolean)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
-                  alias: test2_n0
-                  Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: test1_n2
+                  Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_1 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
                   Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string), key BETWEEN 100 AND 102 (type: boolean)
+                    expressions: key (type: int), value (type: int), col_1 (type: string), key BETWEEN 100 AND 102 (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6 Data size: 596 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col1 (type: int)
                       sort order: +
                       Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 596 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: boolean)
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -2282,20 +2304,20 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Right Outer Join 0 to 1
+                     Left Outer Join 0 to 1
                 keys:
                   0 _col1 (type: int)
                   1 _col1 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                residual filter predicates: {(_col0 BETWEEN 100 AND 102 or _col6)}
-                Statistics: Num rows: 10 Data size: 1850 Basic stats: COMPLETE Column stats: COMPLETE
+                residual filter predicates: {(_col4 BETWEEN 100 AND 102 or _col3)}
+                Statistics: Num rows: 7 Data size: 1261 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
+                  expressions: _col4 (type: int), _col5 (type: int), _col6 (type: string), _col0 (type: int), _col1 (type: int), _col2 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 7 Data size: 1233 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     sort order: 
-                    Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 7 Data size: 1233 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -2308,10 +2330,10 @@ STAGE PLANS:
                   1 
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
                 residual filter predicates: {(_col1 is null or (_col10 is null and (_col7 <> _col4)))}
-                Statistics: Num rows: 110 Data size: 41722 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 63 Data size: 23764 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 110 Data size: 41722 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 63 Data size: 23764 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2327,14 +2349,14 @@ STAGE PLANS:
                   1 _col1 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
-                Statistics: Num rows: 11 Data size: 1947 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     sort order: 
-                    Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
 
   Stage: Stage-0
@@ -2343,7 +2365,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: SELECT *
 FROM (
   SELECT test1_n2.key AS key1, test1_n2.value AS value1, test1_n2.col_1 AS col_1,
diff --git a/ql/src/test/results/clientpositive/llap/join_emit_interval.q.out b/ql/src/test/results/clientpositive/llap/join_emit_interval.q.out
index 05424ad..38bede3 100644
--- a/ql/src/test/results/clientpositive/llap/join_emit_interval.q.out
+++ b/ql/src/test/results/clientpositive/llap/join_emit_interval.q.out
@@ -89,17 +89,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n4
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -115,14 +119,14 @@ STAGE PLANS:
                   0 _col1 (type: int)
                   1 _col1 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6
-                Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 9 Data size: 1521 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/limit_join_transpose.q.out b/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
index b0e8aea..c07ee83 100644
--- a/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
+++ b/ql/src/test/results/clientpositive/llap/limit_join_transpose.q.out
@@ -47,17 +47,21 @@ STAGE PLANS:
             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
-                  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)
-                      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)
+                        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: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -152,17 +156,21 @@ STAGE PLANS:
             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
-                  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)
-                      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)
+                        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: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -254,16 +262,19 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: string), _col1 (type: string)
-                  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)
-                      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)
+                        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: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -366,16 +377,19 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   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)
-                      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)
+                        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)
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
@@ -388,16 +402,19 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: string), _col1 (type: string)
-                  Select Operator
-                    expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (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: _col0 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -528,16 +545,19 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   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)
-                      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)
+                        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)
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
@@ -550,16 +570,19 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: string), _col1 (type: string)
-                  Select Operator
-                    expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (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: _col0 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -707,16 +730,19 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   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)
-                      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)
+                        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)
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
@@ -733,16 +759,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src3
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: value (type: string)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: value (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -963,17 +993,21 @@ STAGE PLANS:
             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
-                  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)
-                      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)
+                        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: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1069,17 +1103,21 @@ STAGE PLANS:
             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
-                  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)
-                      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)
+                        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: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1173,16 +1211,19 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: string), _col1 (type: string)
-                  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)
-                      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)
+                        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: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1286,16 +1327,19 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   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)
-                      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)
+                        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)
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
@@ -1309,16 +1353,19 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: string), _col1 (type: string)
-                  Select Operator
-                    expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (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: _col0 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1450,16 +1497,19 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   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)
-                      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)
+                        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)
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
@@ -1473,16 +1523,19 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: string), _col1 (type: string)
-                  Select Operator
-                    expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (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: _col0 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1633,16 +1686,19 @@ STAGE PLANS:
                 TableScan
                   alias: src1
                   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)
-                      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)
+                        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)
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
@@ -1659,16 +1715,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src3
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: value (type: string)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: value (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
diff --git a/ql/src/test/results/clientpositive/llap/lineage2.q.out b/ql/src/test/results/clientpositive/llap/lineage2.q.out
index 9543864..c0cd91f 100644
--- a/ql/src/test/results/clientpositive/llap/lineage2.q.out
+++ b/ql/src/test/results/clientpositive/llap/lineage2.q.out
@@ -552,7 +552,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Input: default@src1
 PREHOOK: Output: default@dest_l1_n0
-{"version":"1.0","engine":"tez","database":"default","hash":"40b5d904f13549d8c25bd0be758f5b6f","queryText":"INSERT OVERWRITE TABLE dest_l1_n0\nSELECT j.*\nFROM (SELECT t1.key, p1.value\n      FROM src1 t1\n      LEFT OUTER JOIN src p1\n      ON (t1.key = p1.key)\n      UNION ALL\n      SELECT t2.key, p2.value\n      FROM src1 t2\n      LEFT OUTER JOIN src p2\n      ON (t2.key = p2.key)) j","edges":[{"sources":[2],"targets":[0],"expression":"UDFToInteger(j.key)","edgeType":"PROJECTION"},{ [...]
+{"version":"1.0","engine":"tez","database":"default","hash":"40b5d904f13549d8c25bd0be758f5b6f","queryText":"INSERT OVERWRITE TABLE dest_l1_n0\nSELECT j.*\nFROM (SELECT t1.key, p1.value\n      FROM src1 t1\n      LEFT OUTER JOIN src p1\n      ON (t1.key = p1.key)\n      UNION ALL\n      SELECT t2.key, p2.value\n      FROM src1 t2\n      LEFT OUTER JOIN src p2\n      ON (t2.key = p2.key)) j","edges":[{"sources":[2],"targets":[0],"expression":"UDFToInteger(j.key)","edgeType":"PROJECTION"},{ [...]
 PREHOOK: query: drop table if exists emp
 PREHOOK: type: DROPTABLE
 PREHOOK: query: drop table if exists dept_n10
diff --git a/ql/src/test/results/clientpositive/llap/lineage3.q.out b/ql/src/test/results/clientpositive/llap/lineage3.q.out
index b9ef761..783dfa7 100644
--- a/ql/src/test/results/clientpositive/llap/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/llap/lineage3.q.out
@@ -180,13 +180,13 @@ PREHOOK: Input: default@src1
 #### A masked pattern was here ####
 {"version":"1.0","engine":"tez","database":"default","hash":"94e9cc0a67801fe1503a3cb0c5029d59","queryText":"select * from src1 a\nwhere exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > 300.0D)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":" [...]
 311	val_311
-Warning: Shuffle Join MERGEJOIN[36][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select key, value from src1
 where key not in (select key+18 from src1) order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-{"version":"1.0","engine":"tez","database":"default","hash":"cbc4367150807328dda0f1cf4c74b811","queryText":"select key, value from src1\nwhere key not in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + 18.0D))","edgeType":"PREDICATE"},{"sources":[4,2],"targets":[0,1],"expression":"((true [...]
+{"version":"1.0","engine":"tez","database":"default","hash":"cbc4367150807328dda0f1cf4c74b811","queryText":"select key, value from src1\nwhere key not in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + 18.0D))","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"UDFToDou [...]
 PREHOOK: query: select * from src1 a
 where not exists
   (select cint from alltypesorc b
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin3.q.out b/ql/src/test/results/clientpositive/llap/mapjoin3.q.out
index ac36e4f..39af0d7 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin3.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin3.q.out
@@ -147,17 +147,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test_1
+                  filterExpr: member is not null (type: boolean)
                   Statistics: Num rows: 3 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: member (type: bigint), age (type: varchar(100)), age (type: varchar(100))
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: bigint)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: bigint)
+                  Filter Operator
+                    predicate: member is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 282 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: member (type: bigint), age (type: varchar(100)), age (type: varchar(100))
+                      outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: varchar(100)), _col2 (type: varchar(100))
+                      Reduce Output Operator
+                        key expressions: _col0 (type: bigint)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: bigint)
+                        Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: varchar(100)), _col2 (type: varchar(100))
             Execution mode: vectorized, llap
             LLAP IO: no inputs
 
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin46.q.out b/ql/src/test/results/clientpositive/llap/mapjoin46.q.out
index d9d2396..fe648f6 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin46.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin46.q.out
@@ -86,10 +86,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       input vertices:
                         1 Map 2
-                      Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -100,17 +100,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n2
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
 
@@ -213,10 +217,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n2
-                  filterExpr: key BETWEEN 100 AND 102 (type: boolean)
+                  filterExpr: (key BETWEEN 100 AND 102 and value is not null) (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: key BETWEEN 100 AND 102 (type: boolean)
+                    predicate: (key BETWEEN 100 AND 102 and value is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: int), value (type: int), col_2 (type: string)
@@ -408,17 +412,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test1_n4
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_1 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Map 2 
@@ -439,10 +447,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       input vertices:
                         0 Map 1
-                      Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 1237 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
                         compressed: false
-                        Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 1237 Basic stats: COMPLETE Column stats: COMPLETE
                         table:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -983,14 +991,14 @@ STAGE PLANS:
                       input vertices:
                         1 Map 2
                       residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
-                      Statistics: Num rows: 11 Data size: 1947 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1001,17 +1009,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n2
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
 
@@ -1424,17 +1436,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test1_n4
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_1 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Map 2 
@@ -1456,14 +1472,14 @@ STAGE PLANS:
                       input vertices:
                         0 Map 1
                       residual filter predicates: {(_col0 BETWEEN 100 AND 102 or _col6)}
-                      Statistics: Num rows: 10 Data size: 1850 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 7 Data size: 1265 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 1237 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 7 Data size: 1237 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2442,7 +2458,7 @@ POSTHOOK: Input: default@test2_n2
 NULL	NULL	NULL	104	3	Fli
 NULL	NULL	NULL	105	NULL	None
 NULL	NULL	None	NULL	NULL	NULL
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM (
@@ -2498,30 +2514,20 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 4 (BROADCAST_EDGE)
-        Map 3 <- Map 1 (BROADCAST_EDGE)
+        Map 1 <- Map 3 (BROADCAST_EDGE)
+        Map 3 <- Map 4 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: test1_n4
-                  Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  alias: test2_n2
+                  Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string), key BETWEEN 100 AND 102 (type: boolean)
+                    expressions: key (type: int), value (type: int), col_2 (type: string), key BETWEEN 100 AND 102 (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 6 Data size: 596 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
                     Map Join Operator
                       condition map:
                            Left Outer Join 0 to 1
@@ -2530,46 +2536,59 @@ STAGE PLANS:
                         1 _col1 (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       input vertices:
-                        1 Map 4
-                      residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
-                      Statistics: Num rows: 11 Data size: 1947 Basic stats: COMPLETE Column stats: COMPLETE
+                        1 Map 3
+                      residual filter predicates: {(_col4 BETWEEN 100 AND 102 or _col3)}
+                      Statistics: Num rows: 7 Data size: 1261 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
-                        expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
+                        expressions: _col4 (type: int), _col5 (type: int), _col6 (type: string), _col0 (type: int), _col1 (type: int), _col2 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 1233 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           sort order: 
-                          Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 7 Data size: 1233 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Map 3 
             Map Operator Tree:
                 TableScan
-                  alias: test2_n2
-                  Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: test1_n4
+                  Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_1 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
                   Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string), key BETWEEN 100 AND 102 (type: boolean)
+                    expressions: key (type: int), value (type: int), col_1 (type: string), key BETWEEN 100 AND 102 (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6 Data size: 596 Basic stats: COMPLETE Column stats: COMPLETE
                     Map Join Operator
                       condition map:
-                           Right Outer Join 0 to 1
+                           Left Outer Join 0 to 1
                       keys:
                         0 _col1 (type: int)
                         1 _col1 (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       input vertices:
-                        0 Map 1
-                      residual filter predicates: {(_col0 BETWEEN 100 AND 102 or _col6)}
-                      Statistics: Num rows: 10 Data size: 1850 Basic stats: COMPLETE Column stats: COMPLETE
+                        1 Map 4
+                      residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
+                      Statistics: Num rows: 9 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
-                        expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
+                        expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           sort order: 
-                          Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
@@ -2577,17 +2596,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n2
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -2601,10 +2624,10 @@ STAGE PLANS:
                   1 
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
                 residual filter predicates: {(_col1 is null or (_col10 is null and (_col7 <> _col4)))}
-                Statistics: Num rows: 110 Data size: 41722 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 63 Data size: 23764 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 110 Data size: 41722 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 63 Data size: 23764 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2616,7 +2639,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: SELECT *
 FROM (
   SELECT test1_n4.key AS key1, test1_n4.value AS value1, test1_n4.col_1 AS col_1,
@@ -2681,7 +2704,7 @@ NULL	NULL	NULL	105	NULL	None	98	NULL	None	NULL	NULL	NULL
 NULL	NULL	NULL	105	NULL	None	99	0	Alice	NULL	NULL	NULL
 NULL	NULL	NULL	105	NULL	None	99	2	Mat	102	2	Del
 NULL	NULL	NULL	105	NULL	None	NULL	NULL	None	NULL	NULL	NULL
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT *
 FROM (
@@ -2737,30 +2760,20 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Map 1 <- Map 4 (BROADCAST_EDGE)
-        Map 3 <- Map 1 (BROADCAST_EDGE)
+        Map 1 <- Map 3 (BROADCAST_EDGE)
+        Map 3 <- Map 4 (BROADCAST_EDGE)
         Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: test1_n4
-                  Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  alias: test2_n2
+                  Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: key (type: int), value (type: int), col_1 (type: string), key BETWEEN 100 AND 102 (type: boolean)
+                    expressions: key (type: int), value (type: int), col_2 (type: string), key BETWEEN 100 AND 102 (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 6 Data size: 596 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
                     Map Join Operator
                       condition map:
                            Left Outer Join 0 to 1
@@ -2769,46 +2782,59 @@ STAGE PLANS:
                         1 _col1 (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       input vertices:
-                        1 Map 4
-                      residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
-                      Statistics: Num rows: 11 Data size: 1947 Basic stats: COMPLETE Column stats: COMPLETE
+                        1 Map 3
+                      residual filter predicates: {(_col4 BETWEEN 100 AND 102 or _col3)}
+                      Statistics: Num rows: 7 Data size: 1261 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
-                        expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
+                        expressions: _col4 (type: int), _col5 (type: int), _col6 (type: string), _col0 (type: int), _col1 (type: int), _col2 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 7 Data size: 1233 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           sort order: 
-                          Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 7 Data size: 1233 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Map 3 
             Map Operator Tree:
                 TableScan
-                  alias: test2_n2
-                  Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+                  alias: test1_n4
+                  Statistics: Num rows: 6 Data size: 572 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_1 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 4 Data size: 384 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
                   Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string), key BETWEEN 100 AND 102 (type: boolean)
+                    expressions: key (type: int), value (type: int), col_1 (type: string), key BETWEEN 100 AND 102 (type: boolean)
                     outputColumnNames: _col0, _col1, _col2, _col3
-                    Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6 Data size: 596 Basic stats: COMPLETE Column stats: COMPLETE
                     Map Join Operator
                       condition map:
-                           Right Outer Join 0 to 1
+                           Left Outer Join 0 to 1
                       keys:
                         0 _col1 (type: int)
                         1 _col1 (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                       input vertices:
-                        0 Map 1
-                      residual filter predicates: {(_col0 BETWEEN 100 AND 102 or _col6)}
-                      Statistics: Num rows: 10 Data size: 1850 Basic stats: COMPLETE Column stats: COMPLETE
+                        1 Map 4
+                      residual filter predicates: {(_col3 or _col4 BETWEEN 100 AND 102)}
+                      Statistics: Num rows: 9 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
-                        expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
+                        expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
                           sort order: 
-                          Statistics: Num rows: 10 Data size: 1810 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
@@ -2816,17 +2842,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2_n2
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -2840,10 +2870,10 @@ STAGE PLANS:
                   1 
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
                 residual filter predicates: {(_col1 is null or (_col10 is null and (_col7 <> _col4)))}
-                Statistics: Num rows: 110 Data size: 41722 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 63 Data size: 23764 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 110 Data size: 41722 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 63 Data size: 23764 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2855,7 +2885,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: SELECT *
 FROM (
   SELECT test1_n4.key AS key1, test1_n4.value AS value1, test1_n4.col_1 AS col_1,
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_emit_interval.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_emit_interval.q.out
index 8c9008a..3cb69a8 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin_emit_interval.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin_emit_interval.q.out
@@ -89,14 +89,14 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6
                       input vertices:
                         1 Map 2
-                      Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                       Select Operator
                         expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col4 (type: int), _col5 (type: int), _col6 (type: string)
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                         File Output Operator
                           compressed: false
-                          Statistics: Num rows: 11 Data size: 1903 Basic stats: COMPLETE Column stats: COMPLETE
+                          Statistics: Num rows: 9 Data size: 1521 Basic stats: COMPLETE Column stats: COMPLETE
                           table:
                               input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                               output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -107,17 +107,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: test2
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), value (type: int), col_2 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 4 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int), _col2 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: int), col_2 (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 285 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
 
diff --git a/ql/src/test/results/clientpositive/llap/mergejoin.q.out b/ql/src/test/results/clientpositive/llap/mergejoin.q.out
index 61d50b0..6872f9d 100644
--- a/ql/src/test/results/clientpositive/llap/mergejoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/mergejoin.q.out
@@ -1537,11 +1537,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: ###Masked### Data size: ###Masked### Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: ###Masked### Data size: ###Masked### Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: ###Masked### Data size: ###Masked### Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -1671,11 +1675,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: ###Masked### Data size: ###Masked### Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: ###Masked### Data size: ###Masked### Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: ###Masked### Data size: ###Masked### Basic stats: COMPLETE Column stats: COMPLETE
             Map Operator Tree:
                 TableScan
                   alias: b
diff --git a/ql/src/test/results/clientpositive/llap/sharedwork.q.out b/ql/src/test/results/clientpositive/llap/sharedwork.q.out
index 09c10b8..3721a0a 100644
--- a/ql/src/test/results/clientpositive/llap/sharedwork.q.out
+++ b/ql/src/test/results/clientpositive/llap/sharedwork.q.out
@@ -198,6 +198,7 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: table__1232
+                  filterExpr: ((col_24) IN ('part1', 'part2', 'part3') or col_24 is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
                   GatherStats: false
                   Filter Operator
@@ -217,19 +218,23 @@ STAGE PLANS:
                         tag: 1
                         value expressions: _col1 (type: string)
                         auto parallelism: true
-                  Select Operator
-                    expressions: col_24 (type: string), col_21 (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: col_24 is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      null sort order: a
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: col_24 (type: string), col_21 (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
-                      tag: 1
-                      value expressions: _col1 (type: string)
-                      auto parallelism: true
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: a
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                        tag: 1
+                        value expressions: _col1 (type: string)
+                        auto parallelism: true
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Path -> Alias:
@@ -289,21 +294,26 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: table__133
+                  filterExpr: col_1 is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: NONE
                   GatherStats: false
-                  Select Operator
-                    expressions: col_1 (type: string), col_22 (type: string), col_23 (type: int)
-                    outputColumnNames: _col0, _col1, _col2
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: col_1 is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      null sort order: a
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: col_1 (type: string), col_22 (type: string), col_23 (type: int)
+                      outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: NONE
-                      tag: 1
-                      value expressions: _col1 (type: string), _col2 (type: int)
-                      auto parallelism: true
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: a
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: NONE
+                        tag: 1
+                        value expressions: _col1 (type: string), _col2 (type: int)
+                        auto parallelism: true
             Execution mode: vectorized, llap
             LLAP IO: no inputs
             Path -> Alias:
@@ -363,12 +373,12 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: table__1215
-                  filterExpr: (UDFToDouble(col_100) = 210.0D) (type: boolean)
+                  filterExpr: ((UDFToDouble(col_100) = 210.0D) and col_1 is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: (UDFToDouble(col_100) = 210.0D) (type: boolean)
+                    predicate: ((UDFToDouble(col_100) = 210.0D) and col_1 is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: col_1 (type: string)
diff --git a/ql/src/test/results/clientpositive/llap/skewjoinopt15.q.out b/ql/src/test/results/clientpositive/llap/skewjoinopt15.q.out
index 35f7051..ec296ec 100644
--- a/ql/src/test/results/clientpositive/llap/skewjoinopt15.q.out
+++ b/ql/src/test/results/clientpositive/llap/skewjoinopt15.q.out
@@ -203,17 +203,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 6 Data size: 540 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int), val (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 6 Data size: 540 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), val (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 6 Data size: 540 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 6 Data size: 540 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
@@ -425,16 +429,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
diff --git a/ql/src/test/results/clientpositive/llap/smb_mapjoin_17.q.out b/ql/src/test/results/clientpositive/llap/smb_mapjoin_17.q.out
index 544ca58..57a252d 100644
--- a/ql/src/test/results/clientpositive/llap/smb_mapjoin_17.q.out
+++ b/ql/src/test/results/clientpositive/llap/smb_mapjoin_17.q.out
@@ -636,96 +636,120 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
                   alias: d
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 
             Map Operator Tree:
                 TableScan
                   alias: e
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 7 
             Map Operator Tree:
                 TableScan
                   alias: f
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 8 
             Map Operator Tree:
                 TableScan
                   alias: g
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -928,112 +952,140 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
                   alias: d
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 
             Map Operator Tree:
                 TableScan
                   alias: e
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 7 
             Map Operator Tree:
                 TableScan
                   alias: f
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 8 
             Map Operator Tree:
                 TableScan
                   alias: g
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 9 
             Map Operator Tree:
                 TableScan
                   alias: h
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1383,304 +1435,380 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: j
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 11 
             Map Operator Tree:
                 TableScan
                   alias: k
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 12 
             Map Operator Tree:
                 TableScan
                   alias: l
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 13 
             Map Operator Tree:
                 TableScan
                   alias: m
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 14 
             Map Operator Tree:
                 TableScan
                   alias: n
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 15 
             Map Operator Tree:
                 TableScan
                   alias: o
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 16 
             Map Operator Tree:
                 TableScan
                   alias: p
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 17 
             Map Operator Tree:
                 TableScan
                   alias: q
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 18 
             Map Operator Tree:
                 TableScan
                   alias: r
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 19 
             Map Operator Tree:
                 TableScan
                   alias: s
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 2 
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 20 
             Map Operator Tree:
                 TableScan
                   alias: t
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: d
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 5 
             Map Operator Tree:
                 TableScan
                   alias: e
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 6 
             Map Operator Tree:
                 TableScan
                   alias: f
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 7 
             Map Operator Tree:
                 TableScan
                   alias: g
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 8 
             Map Operator Tree:
                 TableScan
                   alias: h
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 9 
             Map Operator Tree:
                 TableScan
                   alias: i
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
 
diff --git a/ql/src/test/results/clientpositive/llap/smb_mapjoin_4.q.out b/ql/src/test/results/clientpositive/llap/smb_mapjoin_4.q.out
index cf0663e..ce3bc24 100644
--- a/ql/src/test/results/clientpositive/llap/smb_mapjoin_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/smb_mapjoin_4.q.out
@@ -324,11 +324,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -356,17 +360,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -441,33 +449,41 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Merge Join Operator
-                      condition map:
-                           Left Outer Join 0 to 1
-                      keys:
-                        0 _col0 (type: int)
-                        1 _col0 (type: int)
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col2 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col2 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                      Merge Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3
                         Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
+                        Reduce Output Operator
+                          key expressions: _col2 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col2 (type: int)
+                          Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
             Execution mode: llap
         Map 4 
             Map Operator Tree:
@@ -559,11 +575,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -808,11 +828,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: b
@@ -840,17 +864,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -924,33 +952,41 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Merge Join Operator
-                      condition map:
-                           Right Outer Join 0 to 1
-                      keys:
-                        0 _col0 (type: int)
-                        1 _col0 (type: int)
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col2 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col2 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                      Merge Join Operator
+                        condition map:
+                             Right Outer Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3
                         Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
+                        Reduce Output Operator
+                          key expressions: _col2 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col2 (type: int)
+                          Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
             Execution mode: llap
         Map 4 
             Map Operator Tree:
@@ -1042,11 +1078,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: b
@@ -1323,17 +1363,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1422,45 +1466,50 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE)
+        Reducer 3 <- Map 2 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
-        Map 1 
+        Map 2 
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
+                      Merge Join Operator
+                        condition map:
+                             Right Outer Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3
+                        Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col2 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col2 (type: int)
+                          Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
+            Execution mode: llap
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: c
@@ -1477,23 +1526,6 @@ STAGE PLANS:
                       value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Reducer 2 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Full Outer Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col2 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col2 (type: int)
-                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/llap/smb_mapjoin_5.q.out b/ql/src/test/results/clientpositive/llap/smb_mapjoin_5.q.out
index fa997c4..df458ba 100644
--- a/ql/src/test/results/clientpositive/llap/smb_mapjoin_5.q.out
+++ b/ql/src/test/results/clientpositive/llap/smb_mapjoin_5.q.out
@@ -324,11 +324,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -356,17 +360,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -441,33 +449,41 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Merge Join Operator
-                      condition map:
-                           Left Outer Join 0 to 1
-                      keys:
-                        0 _col0 (type: int)
-                        1 _col0 (type: int)
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col2 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col2 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                      Merge Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3
                         Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
+                        Reduce Output Operator
+                          key expressions: _col2 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col2 (type: int)
+                          Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
             Execution mode: llap
         Map 4 
             Map Operator Tree:
@@ -559,11 +575,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: a
@@ -808,11 +828,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: b
@@ -840,17 +864,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 3 
@@ -924,33 +952,41 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Merge Join Operator
-                      condition map:
-                           Right Outer Join 0 to 1
-                      keys:
-                        0 _col0 (type: int)
-                        1 _col0 (type: int)
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col2 (type: int)
-                        sort order: +
-                        Map-reduce partition columns: _col2 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                      Merge Join Operator
+                        condition map:
+                             Right Outer Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3
                         Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
+                        Reduce Output Operator
+                          key expressions: _col2 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col2 (type: int)
+                          Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
             Execution mode: llap
         Map 4 
             Map Operator Tree:
@@ -1042,11 +1078,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
             Map Operator Tree:
                 TableScan
                   alias: b
@@ -1323,17 +1363,21 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1422,45 +1466,50 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE)
+        Reducer 3 <- Map 2 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
-        Map 1 
+        Map 2 
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: no inputs
-        Map 5 
+                      Merge Join Operator
+                        condition map:
+                             Right Outer Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3
+                        Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col2 (type: int)
+                          sort order: +
+                          Map-reduce partition columns: _col2 (type: int)
+                          Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
+            Execution mode: llap
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: c
@@ -1477,23 +1526,6 @@ STAGE PLANS:
                       value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
-        Reducer 2 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Full Outer Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  key expressions: _col2 (type: int)
-                  sort order: +
-                  Map-reduce partition columns: _col2 (type: int)
-                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
index a905927..b532699 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
@@ -1638,9 +1638,9 @@ POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@src_null_n4
 POSTHOOK: Lineage: src_null_n4.key SCRIPT []
 POSTHOOK: Lineage: src_null_n4.value EXPRESSION []
-Warning: Map Join MAPJOIN[131][bigTable=?] in task 'Map 1' is a cross product
-Warning: Map Join MAPJOIN[132][bigTable=?] in task 'Map 5' is a cross product
-Warning: Map Join MAPJOIN[133][bigTable=?] in task 'Reducer 6' is a cross product
+Warning: Map Join MAPJOIN[133][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[134][bigTable=?] in task 'Map 5' is a cross product
+Warning: Map Join MAPJOIN[135][bigTable=?] in task 'Reducer 6' is a cross product
 PREHOOK: query: explain
 select key, value, count(*)
 from src_null_n4 b
@@ -1757,40 +1757,44 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: string), value (type: string)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
-                    Map Join Operator
-                      condition map:
-                           Left Outer Join 0 to 1
-                      keys:
-                        0 
-                        1 
-                      outputColumnNames: _col0, _col1, _col2, _col3
-                      input vertices:
-                        1 Reducer 4
-                      residual filter predicates: {(_col2 <> _col1)}
-                      Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
-                      Filter Operator
-                        predicate: _col3 is null (type: boolean)
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Left Outer Join 0 to 1
+                        keys:
+                          0 
+                          1 
+                        outputColumnNames: _col0, _col1, _col2, _col3
+                        input vertices:
+                          1 Reducer 4
+                        residual filter predicates: {(_col2 <> _col1)}
                         Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
-                        Select Operator
-                          expressions: _col0 (type: string), _col1 (type: string)
-                          outputColumnNames: _col0, _col1
+                        Filter Operator
+                          predicate: _col3 is null (type: boolean)
                           Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
-                          Group By Operator
-                            keys: _col1 (type: string), _col0 (type: string)
-                            minReductionHashAggr: 0.99
-                            mode: hash
+                          Select Operator
+                            expressions: _col0 (type: string), _col1 (type: string)
                             outputColumnNames: _col0, _col1
                             Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
-                            Reduce Output Operator
-                              key expressions: _col0 (type: string), _col1 (type: string)
-                              sort order: ++
-                              Map-reduce partition columns: _col0 (type: string)
+                            Group By Operator
+                              keys: _col1 (type: string), _col0 (type: string)
+                              minReductionHashAggr: 0.99
+                              mode: hash
+                              outputColumnNames: _col0, _col1
                               Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                              Reduce Output Operator
+                                key expressions: _col0 (type: string), _col1 (type: string)
+                                sort order: ++
+                                Map-reduce partition columns: _col0 (type: string)
+                                Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
@@ -1992,9 +1996,9 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[131][bigTable=?] in task 'Map 1' is a cross product
-Warning: Map Join MAPJOIN[132][bigTable=?] in task 'Map 5' is a cross product
-Warning: Map Join MAPJOIN[133][bigTable=?] in task 'Reducer 6' is a cross product
+Warning: Map Join MAPJOIN[133][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[134][bigTable=?] in task 'Map 5' is a cross product
+Warning: Map Join MAPJOIN[135][bigTable=?] in task 'Reducer 6' is a cross product
 PREHOOK: query: select key, value, count(*)
 from src_null_n4 b
 where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value)
diff --git a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
index a6b8885..10858c2 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
@@ -260,7 +260,7 @@ POSTHOOK: Input: default@part_null
 85768	almond antique chartreuse lavender yellow	Manufacturer#1	Brand#12	LARGE BRUSHED STEEL	34	SM BAG	1753.76	refull
 86428	almond aquamarine burnished black steel	Manufacturer#1	Brand#12	STANDARD ANODIZED STEEL	28	WRAP BAG	1414.42	arefully 
 90681	almond antique chartreuse khaki white	Manufacturer#3	Brand#31	MEDIUM BURNISHED TIN	17	SM CASE	1671.68	are slyly after the sl
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
 PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND p_brand NOT IN (select p_name from part_null)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part_null
@@ -312,17 +312,20 @@ STAGE PLANS:
                 TableScan
                   alias: part_null
                   Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-                  Group By Operator
-                    keys: p_name (type: string)
-                    minReductionHashAggr: 0.99
-                    mode: hash
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: p_name is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Group By Operator
+                      keys: p_name (type: string)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: p_name (type: string)
                     outputColumnNames: p_name
@@ -465,7 +468,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
 PREHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND p_brand NOT IN (select p_name from part_null)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part_null
@@ -474,7 +477,7 @@ POSTHOOK: query: select * from part_null where p_name IN (select p_name from par
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part_null
 #### A masked pattern was here ####
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
 PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND p_brand NOT IN (select p_type from part_null)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part_null
@@ -526,17 +529,20 @@ STAGE PLANS:
                 TableScan
                   alias: part_null
                   Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-                  Group By Operator
-                    keys: p_type (type: string)
-                    minReductionHashAggr: 0.99
-                    mode: hash
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: p_type is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Group By Operator
+                      keys: p_type (type: string)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: p_type (type: string)
                     outputColumnNames: p_type
@@ -687,7 +693,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
 PREHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND p_brand NOT IN (select p_type from part_null)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part_null
@@ -947,7 +953,7 @@ POSTHOOK: query: select * from part_null where p_brand IN (select p_brand from p
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part_null
 #### A masked pattern was here ####
-Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part_null where p_name NOT IN (select c from tempty) AND p_brand IN (select p_brand from part_null)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part_null
@@ -1000,17 +1006,20 @@ STAGE PLANS:
                 TableScan
                   alias: tempty
                   Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE
-                  Group By Operator
-                    keys: c (type: char(2))
-                    minReductionHashAggr: 0.99
-                    mode: hash
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: CAST( c AS STRING) is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: char(2))
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: char(2))
+                    Group By Operator
+                      keys: c (type: char(2))
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(2))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: char(2))
+                        Statistics: Num rows: 1 Data size: 86 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: c (type: char(2))
                     outputColumnNames: c
@@ -1151,7 +1160,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[59][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part_null where p_name NOT IN (select c from tempty) AND p_brand IN (select p_brand from part_null)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part_null
@@ -3043,7 +3052,7 @@ POSTHOOK: Input: default@part_null
 85768	almond antique chartreuse lavender yellow	Manufacturer#1	Brand#12	LARGE BRUSHED STEEL	34	SM BAG	1753.76	refull
 86428	almond aquamarine burnished black steel	Manufacturer#1	Brand#12	STANDARD ANODIZED STEEL	28	WRAP BAG	1414.42	arefully 
 90681	almond antique chartreuse khaki white	Manufacturer#3	Brand#31	MEDIUM BURNISHED TIN	17	SM CASE	1671.68	are slyly after the sl
-Warning: Shuffle Join MERGEJOIN[83][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[85][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
 PREHOOK: query: explain select p.p_partkey, li.l_suppkey
 from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey
 where li.l_linenumber = 1 and
@@ -3262,16 +3271,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: (_col0 / _col1) (type: double), true (type: boolean)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: double)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: double)
+                Filter Operator
+                  predicate: (_col0 is not null and _col1 is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: (_col0 / _col1) (type: double), true (type: boolean)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: double)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: double)
+                      Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: boolean)
                 Select Operator
                   expressions: (_col0 / _col1) (type: double)
                   outputColumnNames: _col0
@@ -3292,7 +3304,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[83][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[85][tables = [$hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 7' is a cross product
 PREHOOK: query: select p.p_partkey, li.l_suppkey
 from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey
 where li.l_linenumber = 1 and
@@ -4054,7 +4066,7 @@ POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@tnull
 POSTHOOK: Lineage: tnull.c SCRIPT []
 POSTHOOK: Lineage: tnull.i SCRIPT []
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part where p_partkey = 3 OR p_size NOT IN (select i from tnull)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4102,17 +4114,20 @@ STAGE PLANS:
                 TableScan
                   alias: tnull
                   Statistics: Num rows: 3 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: i (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: i is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Group By Operator
+                      keys: i (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: i (type: int)
                     outputColumnNames: i
@@ -4210,7 +4225,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part where p_partkey = 3 OR p_size NOT IN (select i from tnull)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
index c21810c..e55396f 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
@@ -591,7 +591,7 @@ Manufacturer#4	almond azure aquamarine papaya violet	12
 Manufacturer#5	almond antique blue firebrick mint	31
 Manufacturer#5	almond aquamarine dodger light gainsboro	46
 Manufacturer#5	almond azure blanched chiffon midnight	23
-Warning: Shuffle Join MERGEJOIN[46][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[48][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain
 select p_name, p_size 
 from 
@@ -795,16 +795,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: (UDFToDouble(_col0) / _col1) (type: double), true (type: boolean)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: double)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: double)
+                Filter Operator
+                  predicate: (UDFToDouble(_col0) is not null and _col1 is not null) (type: boolean)
+                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: (UDFToDouble(_col0) / _col1) (type: double), true (type: boolean)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: double)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: double)
+                      Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: boolean)
         Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -833,7 +836,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[48][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[50][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select p_name, p_size 
 from 
 part where part.p_size not in 
@@ -1837,7 +1840,7 @@ POSTHOOK: Input: default@part
 85768	almond antique chartreuse lavender yellow	Manufacturer#1	Brand#12	LARGE BRUSHED STEEL	34	SM BAG	1753.76	refull
 86428	almond aquamarine burnished black steel	Manufacturer#1	Brand#12	STANDARD ANODIZED STEEL	28	WRAP BAG	1414.42	arefully 
 90681	almond antique chartreuse khaki white	Manufacturer#3	Brand#31	MEDIUM BURNISHED TIN	17	SM CASE	1671.68	are slyly after the sl
-Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part  where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_partkey
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -1973,17 +1976,20 @@ STAGE PLANS:
                   expressions: _col1 (type: int)
                   outputColumnNames: _col1
                   Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col1 (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
+                    Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: _col1 (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(), count(_col1)
                     minReductionHashAggr: 0.0
@@ -2031,7 +2037,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part  where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_partkey
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2056,7 +2062,7 @@ POSTHOOK: Input: default@part
 78486	almond azure blanched chiffon midnight	Manufacturer#5	Brand#52	LARGE BRUSHED BRASS	23	MED BAG	1464.48	hely blith
 85768	almond antique chartreuse lavender yellow	Manufacturer#1	Brand#12	LARGE BRUSHED STEEL	34	SM BAG	1753.76	refull
 90681	almond antique chartreuse khaki white	Manufacturer#3	Brand#31	MEDIUM BURNISHED TIN	17	SM CASE	1671.68	are slyly after the sl
-Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part where (p_partkey*p_size) NOT IN (select min(p_partkey) from part group by p_type)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2179,17 +2185,20 @@ STAGE PLANS:
                   expressions: _col1 (type: int)
                   outputColumnNames: _col1
                   Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col1 (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
+                    Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: _col1 (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(), count(_col1)
                     minReductionHashAggr: 0.0
@@ -2237,7 +2246,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part where (p_partkey*p_size) NOT IN (select min(p_partkey) from part group by p_type)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2512,7 +2521,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 26
-Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part  where floor(p_retailprice) NOT IN (select floor(min(p_retailprice)) from part group by p_type)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2632,20 +2641,31 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
+                  expressions: _col1 (type: double)
+                  outputColumnNames: _col1
+                  Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
+                    Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: floor(_col1) (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: bigint)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: bigint)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: bigint)
+                          Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
                   expressions: floor(_col1) (type: bigint)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: bigint)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: bigint)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: bigint)
-                      Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(), count(_col0)
                     minReductionHashAggr: 0.0
@@ -2693,7 +2713,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[38][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[39][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part  where floor(p_retailprice) NOT IN (select floor(min(p_retailprice)) from part group by p_type)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -3557,7 +3577,7 @@ almond aquamarine sandy cyan gainsboro
 almond aquamarine yellow dodger mint
 almond azure aquamarine papaya violet
 almond azure blanched chiffon midnight
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select key, count(*) from src where value NOT IN (select key from src) group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -3606,17 +3626,20 @@ STAGE PLANS:
                 TableScan
                   alias: src
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: key (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: key (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 21750 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: key (type: string)
                     outputColumnNames: key
@@ -3805,7 +3828,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select key, count(*) from src where value NOT IN (select key from src) group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -4146,7 +4169,7 @@ POSTHOOK: Input: default@src
 5	3
 70	3
 90	3
-Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part  where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4282,17 +4305,20 @@ STAGE PLANS:
                   expressions: _col1 (type: int)
                   outputColumnNames: _col1
                   Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col1 (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
+                    Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: _col1 (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(), count(_col1)
                     minReductionHashAggr: 0.0
@@ -4340,7 +4366,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part  where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4365,7 +4391,7 @@ POSTHOOK: Input: default@part
 78486	almond azure blanched chiffon midnight	Manufacturer#5	Brand#52	LARGE BRUSHED BRASS	23	MED BAG	1464.48	hely blith
 85768	almond antique chartreuse lavender yellow	Manufacturer#1	Brand#12	LARGE BRUSHED STEEL	34	SM BAG	1753.76	refull
 90681	almond antique chartreuse khaki white	Manufacturer#3	Brand#31	MEDIUM BURNISHED TIN	17	SM CASE	1671.68	are slyly after the sl
-Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part  where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand, p_partkey limit 4
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4505,17 +4531,20 @@ STAGE PLANS:
                   expressions: _col1 (type: int)
                   outputColumnNames: _col1
                   Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col1 (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
+                    Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: _col1 (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
                     aggregations: count(), count(_col1)
                     minReductionHashAggr: 0.0
@@ -4563,7 +4592,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part  where (p_size-1) NOT IN (select min(p_size) from part group by p_type) order by p_brand, p_partkey limit 4
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4576,7 +4605,7 @@ POSTHOOK: Input: default@part
 110592	almond antique salmon chartreuse burlywood	Manufacturer#1	Brand#15	PROMO BURNISHED NICKEL	6	JUMBO PKG	1602.59	 to the furiously
 65667	almond aquamarine pink moccasin thistle	Manufacturer#1	Brand#12	LARGE BURNISHED STEEL	42	JUMBO CASE	1632.66	e across the expr
 85768	almond antique chartreuse lavender yellow	Manufacturer#1	Brand#12	LARGE BRUSHED STEEL	34	SM BAG	1753.76	refull
-Warning: Shuffle Join MERGEJOIN[45][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from src where key NOT IN (select p_name from part UNION ALL select p_brand from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4648,44 +4677,52 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
+                  filterExpr: p_name is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_name (type: string)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: p_name is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: _col0 (type: string)
-                      minReductionHashAggr: 0.0
-                      mode: hash
+                    Select Operator
+                      expressions: p_name (type: string)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: string)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0
                         Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: string)
+                          Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 7 
             Map Operator Tree:
                 TableScan
                   alias: part
+                  filterExpr: p_brand is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_brand (type: string)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: p_brand is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: _col0 (type: string)
-                      minReductionHashAggr: 0.0
-                      mode: hash
+                    Select Operator
+                      expressions: p_brand (type: string)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 26 Data size: 2392 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: string)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0
                         Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: string)
+                          Statistics: Num rows: 25 Data size: 3025 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 8 
@@ -4794,7 +4831,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[45][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from src where key NOT IN (select p_name from part UNION ALL select p_brand from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -5390,10 +5427,10 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: e
-                  filterExpr: UDFToDouble((p_size + 100)) is not null (type: boolean)
+                  filterExpr: (UDFToDouble((p_size + 100)) is not null and p_size is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble((p_size + 100)) is not null (type: boolean)
+                    predicate: (UDFToDouble((p_size + 100)) is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       keys: p_size (type: int)
@@ -5581,7 +5618,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@t2_n0
 POSTHOOK: Lineage: t2_n0.c1 SCRIPT []
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1_n0
@@ -5628,17 +5665,20 @@ STAGE PLANS:
                 TableScan
                   alias: t2_n0
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: c1 (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: c1 is not null (type: boolean)
+                    Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: c1 (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: c1 (type: int)
                     outputColumnNames: c1
@@ -5736,7 +5776,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1_n0
@@ -5830,11 +5870,11 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: t1_n0
-                  filterExpr: UDFToDouble(c2) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(c2) is not null and c2 is not null) (type: boolean)
                   Statistics: Num rows: 4 Data size: 352 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(c2) is not null (type: boolean)
-                    Statistics: Num rows: 3 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (UDFToDouble(c2) is not null and c2 is not null) (type: boolean)
+                    Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       keys: c2 (type: char(100))
                       minReductionHashAggr: 0.0
@@ -6929,7 +6969,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t_n0
 #### A masked pattern was here ####
 7
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select t_n0.i from t_n0 where t_n0.j NOT IN (select t1_n0.i from t_n0 t1_n0 )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t_n0
@@ -6975,17 +7015,20 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n0
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: i (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: i is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: i (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: i (type: int)
                     outputColumnNames: i
@@ -7083,7 +7126,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select t_n0.i from t_n0 where t_n0.j NOT IN (select t1_n0.i from t_n0 t1_n0 )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t_n0
@@ -7094,7 +7137,7 @@ POSTHOOK: Input: default@t_n0
 #### A masked pattern was here ####
 1
 4
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select t_n0.i from t_n0 where t_n0.i NOT IN (select t1_n0.i from t_n0 t1_n0 )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t_n0
@@ -7123,17 +7166,20 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n0
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: i (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: i is not null (type: boolean)
+                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: i (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: i (type: int)
                     outputColumnNames: _col0
@@ -7240,7 +7286,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select t_n0.i from t_n0 where t_n0.i NOT IN (select t1_n0.i from t_n0 t1_n0 )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t_n0
@@ -7253,7 +7299,7 @@ PREHOOK: query: drop table t1_n0
 PREHOOK: type: DROPTABLE
 POSTHOOK: query: drop table t1_n0
 POSTHOOK: type: DROPTABLE
-Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[54][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select *
 from src b
 where b.key not in
@@ -7337,18 +7383,22 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: value is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: value (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: value (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -7475,7 +7525,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[54][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select *
 from src b
 where b.key not in
diff --git a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
index c4f8307..23b5344 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
@@ -3439,8 +3439,8 @@ POSTHOOK: Input: default@part
 85768
 86428
 90681
-Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
-Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select * from part_null_n0 where p_name NOT LIKE (select min(p_name) from part_null_n0) AND p_brand NOT IN (select p_name from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -3473,17 +3473,20 @@ STAGE PLANS:
                 TableScan
                   alias: part
                   Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: p_name (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: p_name is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: p_name (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_name (type: string)
                     outputColumnNames: p_name
@@ -3647,8 +3650,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
-Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_2, $hdt$_3]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select * from part_null_n0 where p_name NOT LIKE (select min(p_name) from part_null_n0) AND p_brand NOT IN (select p_name from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -3684,7 +3687,7 @@ POSTHOOK: Input: default@part_null_n0
 85768	almond antique chartreuse lavender yellow	Manufacturer#1	Brand#12	LARGE BRUSHED STEEL	34	SM BAG	1753.76	refull
 86428	almond aquamarine burnished black steel	Manufacturer#1	Brand#12	STANDARD ANODIZED STEEL	28	WRAP BAG	1414.42	arefully 
 90681	almond antique chartreuse khaki white	Manufacturer#3	Brand#31	MEDIUM BURNISHED TIN	17	SM CASE	1671.68	are slyly after the sl
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select * from part_null_n0 where p_brand NOT IN (select p_name from part) AND p_name NOT LIKE (select min(p_name) from part_null_n0 pp where part_null_n0.p_type = pp.p_type)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -3717,17 +3720,20 @@ STAGE PLANS:
                 TableScan
                   alias: part
                   Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: p_name (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: p_name is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: p_name (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_name (type: string)
                     outputColumnNames: p_name
@@ -3911,7 +3917,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select * from part_null_n0 where p_brand NOT IN (select p_name from part) AND p_name NOT LIKE (select min(p_name) from part_null_n0 pp where part_null_n0.p_type = pp.p_type)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -6241,7 +6247,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select * from part where p_size <>
     (select count(p_size) from part pp where part.p_type <> pp.p_type)
 PREHOOK: type: QUERY
@@ -6303,18 +6309,22 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: part
+                  filterExpr: p_type is not null (type: boolean)
                   Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: p_type (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: p_type is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: p_type (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 13 Data size: 1352 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -6409,7 +6419,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select * from part where p_size <>
     (select count(p_size) from part pp where part.p_type <> pp.p_type)
 PREHOOK: type: QUERY
@@ -6463,7 +6473,7 @@ POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@t_n11
 POSTHOOK: Lineage: t_n11.i SCRIPT []
 POSTHOOK: Lineage: t_n11.j SCRIPT []
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select * from t_n11 where 0 = (select count(*) from t_n11 tt_n11 where tt_n11.j <> t_n11.i)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t_n11
@@ -6523,18 +6533,22 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: t_n11
+                  filterExpr: i is not null (type: boolean)
                   Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: i (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: i is not null (type: boolean)
+                    Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: i (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -6629,7 +6643,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[33][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select * from t_n11 where 0 = (select count(*) from t_n11 tt_n11 where tt_n11.j <> t_n11.i)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t_n11
diff --git a/ql/src/test/results/clientpositive/llap/subquery_select.q.out b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
index 0a01852..e8f489e 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_select.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
@@ -1,4 +1,4 @@
-Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain SELECT p_size, p_size IN (
         SELECT MAX(p_size) FROM part)
 FROM part
@@ -116,16 +116,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), true (type: boolean)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: int), true (type: boolean)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: boolean)
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -154,7 +157,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: SELECT p_size, p_size IN (
         SELECT MAX(p_size) FROM part)
 FROM part
@@ -400,7 +403,7 @@ POSTHOOK: Input: default@part
 6	true
 6	true
 7	true
-Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain SELECT p_size, p_size NOT IN (
         SELECT MAX(p_size) FROM part)
 FROM part
@@ -518,16 +521,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), true (type: boolean)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: int), true (type: boolean)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: boolean)
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -556,7 +562,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[32][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: SELECT p_size, p_size NOT IN (
         SELECT MAX(p_size) FROM part)
 FROM part
@@ -2226,7 +2232,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 POSTHOOK: Output: default@tnull_n3
 POSTHOOK: Lineage: tnull_n3.i EXPRESSION []
-Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select p_size, p_size IN (select i from tnull_n3) from part
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2274,17 +2280,20 @@ STAGE PLANS:
                 TableScan
                   alias: tnull_n3
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: i (type: int)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: i is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Group By Operator
+                      keys: i (type: int)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: i (type: int)
                     outputColumnNames: i
@@ -2379,7 +2388,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[30][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[31][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select p_size, p_size IN (select i from tnull_n3) from part
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -3193,7 +3202,7 @@ STANDARD ANODIZED TIN	true
 STANDARD BURNISHED TIN	true
 STANDARD PLATED TIN	true
 STANDARD POLISHED STEEL	true
-Warning: Shuffle Join MERGEJOIN[73][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[75][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: EXPLAIN SELECT p_size, p_size IN (
         SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) AND
         p_name IN (SELECT min(p_name) from part)
@@ -3404,16 +3413,19 @@ STAGE PLANS:
                 mode: mergepartial
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), true (type: boolean)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
+                Filter Operator
+                  predicate: _col0 is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: string), true (type: boolean)
+                    outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: boolean)
                 Group By Operator
                   aggregations: count(), count(_col0)
                   mode: complete
@@ -3434,7 +3446,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[73][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[75][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: SELECT p_size, p_size IN (
         SELECT MAX(p_size) FROM part p where p.p_type = part.p_type) AND
         p_name IN (SELECT min(p_name) from part)
@@ -3620,8 +3632,8 @@ POSTHOOK: Input: default@part
 6	false
 6	false
 7	false
-Warning: Shuffle Join MERGEJOIN[50][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
-Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[52][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[55][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select p_size, (p_size IN
     (select (select max(p_size) from part) as sb from part order by sb limit 1)) = true
    from part
@@ -3773,16 +3785,19 @@ STAGE PLANS:
                 Limit
                   Number of rows: 1
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int), true (type: boolean)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: _col0 is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: _col0 (type: int), true (type: boolean)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: boolean)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: boolean)
                   Group By Operator
                     aggregations: count(), count(_col0)
                     minReductionHashAggr: 0.0
@@ -3816,8 +3831,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[50][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
-Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[52][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[55][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select p_size, (p_size IN
     (select (select max(p_size) from part) as sb from part order by sb limit 1)) = true
    from part
@@ -4275,7 +4290,7 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part
 #### A masked pattern was here ####
 true
-Warning: Shuffle Join MERGEJOIN[52][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: explain select o.p_size, (select count(distinct p_type) from part p where p.p_partkey = o.p_partkey) tmp
     FROM part o right join (select * from part where p_size > (select avg(p_size) from part)) t on t.p_partkey = o.p_partkey
 PREHOOK: type: QUERY
@@ -4307,16 +4322,19 @@ STAGE PLANS:
                 TableScan
                   alias: o
                   Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_partkey (type: int), p_size (type: int)
-                    outputColumnNames: _col0, _col1
+                  Filter Operator
+                    predicate: p_partkey is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: p_partkey (type: int), p_size (type: int)
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: int)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: int)
                   Select Operator
                     expressions: p_partkey (type: int), UDFToDouble(p_size) (type: double)
                     outputColumnNames: _col0, _col1
@@ -4477,7 +4495,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[52][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[53][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 4' is a cross product
 PREHOOK: query: select o.p_size, (select count(distinct p_type) from part p where p.p_partkey = o.p_partkey) tmp
     FROM part o right join (select * from part where p_size > (select avg(p_size) from part)) t on t.p_partkey = o.p_partkey
 PREHOOK: type: QUERY
diff --git a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_3.q.out b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_3.q.out
index 8caf6c7..4866a42 100644
--- a/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_dynpart_hashjoin_3.q.out
@@ -52,16 +52,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: UDFToInteger(key) is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: UDFToInteger(key) (type: int)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: UDFToInteger(key) (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -164,16 +168,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: UDFToInteger(key) is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: UDFToInteger(key) (type: int)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: UDFToInteger(key) (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -269,16 +277,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: UDFToInteger(key) is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: UDFToInteger(key) (type: int)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (type: boolean)
+                    Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: UDFToInteger(key) (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
diff --git a/ql/src/test/results/clientpositive/llap/tez_fixed_bucket_pruning.q.out b/ql/src/test/results/clientpositive/llap/tez_fixed_bucket_pruning.q.out
index 1574565..e3bebbc 100644
--- a/ql/src/test/results/clientpositive/llap/tez_fixed_bucket_pruning.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_fixed_bucket_pruning.q.out
@@ -468,7 +468,7 @@ FROM (SELECT `t4`.`plan_key`, `t6`.`project_key`
 FROM (SELECT `t3`.`l3_snapshot_number`, `t2`.`plan_detail_object_id`, `t0`.`l3_snapshot_number` AS `l3_snapshot_number0`, `t0`.`plan_key`, `t0`.`finplan_detail_object_id`
 FROM (SELECT `l3_snapshot_number`, `plan_key`, `finplan_detail_object_id`
 FROM `default`.`l3_monthly_dw_dimplan`
-WHERE `idp_data_date` = DATE '2017-12-28') AS `t0`
+WHERE `idp_data_date` = DATE '2017-12-28' AND `finplan_detail_object_id` IS NOT NULL AND `l3_snapshot_number` IS NOT NULL) AS `t0`
 RIGHT JOIN ((SELECT `plan_detail_object_id`
 FROM `default`.`l3_clarity__l3_monthly_dw_factplan_dw_stg_2018022300104_1`
 WHERE `project_object_id` = 7147200) AS `t2`,
@@ -476,7 +476,7 @@ WHERE `project_object_id` = 7147200) AS `t2`,
 FROM `default`.`l3_clarity__l3_snap_number_2018022300104`) AS `t3`) ON `t0`.`finplan_detail_object_id` = `t2`.`plan_detail_object_id` AND `t0`.`l3_snapshot_number` = `t3`.`l3_snapshot_number`) AS `t4`
 LEFT JOIN (SELECT `project_key`, `l3_snapshot_number`
 FROM `default`.`l3_clarity__l3_monthly_dw_factplan_datajoin_1_s2_2018022300104_1`
-WHERE `idp_data_date` = DATE '2017-12-28' AND `project_object_id` = 7147200) AS `t6` ON `t4`.`l3_snapshot_number` = `t6`.`l3_snapshot_number`
+WHERE `idp_data_date` = DATE '2017-12-28' AND `project_object_id` = 7147200 AND `l3_snapshot_number` IS NOT NULL) AS `t6` ON `t4`.`l3_snapshot_number` = `t6`.`l3_snapshot_number`
 ORDER BY `t4`.`plan_key`, `t6`.`project_key`
 LIMIT 5) AS `t9`
 STAGE DEPENDENCIES:
@@ -691,12 +691,12 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  filterExpr: (idp_data_date = DATE'2017-12-28') (type: boolean)
+                  filterExpr: ((idp_data_date = DATE'2017-12-28') and finplan_detail_object_id is not null and l3_snapshot_number is not null) (type: boolean)
                   Statistics: Num rows: 180340 Data size: 14427200 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: (idp_data_date = DATE'2017-12-28') (type: boolean)
+                    predicate: ((idp_data_date = DATE'2017-12-28') and finplan_detail_object_id is not null and l3_snapshot_number is not null) (type: boolean)
                     Statistics: Num rows: 90170 Data size: 7213600 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: l3_snapshot_number (type: bigint), plan_key (type: bigint), finplan_detail_object_id (type: bigint)
@@ -772,12 +772,12 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s2
-                  filterExpr: ((idp_data_date = DATE'2017-12-28') and (project_object_id = 7147200L)) (type: boolean)
+                  filterExpr: ((idp_data_date = DATE'2017-12-28') and (project_object_id = 7147200L) and l3_snapshot_number is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: ((idp_data_date = DATE'2017-12-28') and (project_object_id = 7147200L)) (type: boolean)
+                    predicate: ((idp_data_date = DATE'2017-12-28') and (project_object_id = 7147200L) and l3_snapshot_number is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: project_key (type: bigint), l3_snapshot_number (type: bigint)
@@ -977,7 +977,7 @@ FROM (SELECT `t4`.`plan_key`, `t6`.`project_key`
 FROM (SELECT `t3`.`l3_snapshot_number`, `t2`.`plan_detail_object_id`, `t0`.`l3_snapshot_number` AS `l3_snapshot_number0`, `t0`.`plan_key`, `t0`.`finplan_detail_object_id`
 FROM (SELECT `l3_snapshot_number`, `plan_key`, `finplan_detail_object_id`
 FROM `default`.`l3_monthly_dw_dimplan`
-WHERE `idp_data_date` = DATE '2017-12-28') AS `t0`
+WHERE `idp_data_date` = DATE '2017-12-28' AND `finplan_detail_object_id` IS NOT NULL AND `l3_snapshot_number` IS NOT NULL) AS `t0`
 RIGHT JOIN ((SELECT `plan_detail_object_id`
 FROM `default`.`l3_clarity__l3_monthly_dw_factplan_dw_stg_2018022300104_1`
 WHERE `project_object_id` = 7147200) AS `t2`,
@@ -985,7 +985,7 @@ WHERE `project_object_id` = 7147200) AS `t2`,
 FROM `default`.`l3_clarity__l3_snap_number_2018022300104`) AS `t3`) ON `t0`.`finplan_detail_object_id` = `t2`.`plan_detail_object_id` AND `t0`.`l3_snapshot_number` = `t3`.`l3_snapshot_number`) AS `t4`
 LEFT JOIN (SELECT `project_key`, `l3_snapshot_number`
 FROM `default`.`l3_clarity__l3_monthly_dw_factplan_datajoin_1_s2_2018022300104_1`
-WHERE `idp_data_date` = DATE '2017-12-28' AND `project_object_id` = 7147200) AS `t6` ON `t4`.`l3_snapshot_number` = `t6`.`l3_snapshot_number`
+WHERE `idp_data_date` = DATE '2017-12-28' AND `project_object_id` = 7147200 AND `l3_snapshot_number` IS NOT NULL) AS `t6` ON `t4`.`l3_snapshot_number` = `t6`.`l3_snapshot_number`
 ORDER BY `t4`.`plan_key`, `t6`.`project_key`
 LIMIT 5) AS `t9`
 STAGE DEPENDENCIES:
@@ -1200,13 +1200,12 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  filterExpr: (idp_data_date = DATE'2017-12-28') (type: boolean)
-                  buckets included: [50,] of 64
+                  filterExpr: ((idp_data_date = DATE'2017-12-28') and finplan_detail_object_id is not null and l3_snapshot_number is not null) (type: boolean)
                   Statistics: Num rows: 180340 Data size: 14427200 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: (idp_data_date = DATE'2017-12-28') (type: boolean)
+                    predicate: ((idp_data_date = DATE'2017-12-28') and finplan_detail_object_id is not null and l3_snapshot_number is not null) (type: boolean)
                     Statistics: Num rows: 90170 Data size: 7213600 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: l3_snapshot_number (type: bigint), plan_key (type: bigint), finplan_detail_object_id (type: bigint)
@@ -1282,12 +1281,12 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s2
-                  filterExpr: ((idp_data_date = DATE'2017-12-28') and (project_object_id = 7147200L)) (type: boolean)
+                  filterExpr: ((idp_data_date = DATE'2017-12-28') and (project_object_id = 7147200L) and l3_snapshot_number is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: ((idp_data_date = DATE'2017-12-28') and (project_object_id = 7147200L)) (type: boolean)
+                    predicate: ((idp_data_date = DATE'2017-12-28') and (project_object_id = 7147200L) and l3_snapshot_number is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 80 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: project_key (type: bigint), l3_snapshot_number (type: bigint)
diff --git a/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out b/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
index 0bddcf7..230cd24 100644
--- a/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_join_tests.q.out
@@ -29,16 +29,19 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   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 and value 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)
-                      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)
+                        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)
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
@@ -55,16 +58,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 25 Data size: 2150 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: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -72,17 +79,17 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Right Outer Join 0 to 1
+                     Inner Join 0 to 1
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 64 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col1 (type: string)
-                  Statistics: Num rows: 64 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -94,11 +101,11 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 604 Data size: 124027 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 563 Data size: 111606 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
-                  Statistics: Num rows: 604 Data size: 124027 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 563 Data size: 111606 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -106,10 +113,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 604 Data size: 124027 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 563 Data size: 111606 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 604 Data size: 124027 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 563 Data size: 111606 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/tez_joins_explain.q.out b/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
index d97ad53..79c5640 100644
--- a/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_joins_explain.q.out
@@ -29,16 +29,19 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   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 and value 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)
-                      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)
+                        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)
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
@@ -55,16 +58,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: a
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 25 Data size: 2150 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: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                    Select Operator
+                      expressions: key (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
@@ -72,17 +79,17 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Right Outer Join 0 to 1
+                     Inner Join 0 to 1
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 64 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col1 (type: string)
                   sort order: +
                   Map-reduce partition columns: _col1 (type: string)
-                  Statistics: Num rows: 64 Data size: 7120 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 39 Data size: 6942 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -94,11 +101,11 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 604 Data size: 124027 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 563 Data size: 111606 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
                   sort order: +
-                  Statistics: Num rows: 604 Data size: 124027 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 563 Data size: 111606 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string)
         Reducer 4 
             Execution mode: vectorized, llap
@@ -106,10 +113,10 @@ STAGE PLANS:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 604 Data size: 124027 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 563 Data size: 111606 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 604 Data size: 124027 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 563 Data size: 111606 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/tez_nway_join.q.out b/ql/src/test/results/clientpositive/llap/tez_nway_join.q.out
index 74d76ec..48be427 100644
--- a/ql/src/test/results/clientpositive/llap/tez_nway_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_nway_join.q.out
@@ -348,32 +348,40 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 5 
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -487,32 +495,40 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: b
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: c
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                  Select Operator
-                    expressions: key (type: int)
-                    outputColumnNames: _col0
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
diff --git a/ql/src/test/results/clientpositive/llap/tez_smb_empty.q.out b/ql/src/test/results/clientpositive/llap/tez_smb_empty.q.out
index 771d8f9..57f8f33 100644
--- a/ql/src/test/results/clientpositive/llap/tez_smb_empty.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_smb_empty.q.out
@@ -247,11 +247,15 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s3
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: PARTIAL
-                  Select Operator
-                    expressions: key (type: int), value (type: string), ds (type: string)
-                    outputColumnNames: _col0, _col1, _col2
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
                     Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: PARTIAL
+                    Select Operator
+                      expressions: key (type: int), value (type: string), ds (type: string)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE Column stats: PARTIAL
             Map Operator Tree:
                 TableScan
                   alias: s1
diff --git a/ql/src/test/results/clientpositive/llap/tez_smb_reduce_side.q.out b/ql/src/test/results/clientpositive/llap/tez_smb_reduce_side.q.out
index 90326ca..9bbb181 100644
--- a/ql/src/test/results/clientpositive/llap/tez_smb_reduce_side.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_smb_reduce_side.q.out
@@ -87,20 +87,24 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: x
+                  filterExpr: key is not null (type: boolean)
                   Statistics: Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    aggregations: count(value)
-                    keys: key (type: string)
-                    minReductionHashAggr: 0.0
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 2 Data size: 356 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count(value)
+                      keys: key (type: string)
+                      minReductionHashAggr: 0.0
+                      mode: hash
+                      outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: bigint)
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 95 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 3 
diff --git a/ql/src/test/results/clientpositive/llap/tez_union.q.out b/ql/src/test/results/clientpositive/llap/tez_union.q.out
index 5224b24..bae4176 100644
--- a/ql/src/test/results/clientpositive/llap/tez_union.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_union.q.out
@@ -1295,23 +1295,27 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s
+                  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)
-                      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 Output Operator
-                      key expressions: _col0 (type: string)
-                      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)
+                        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 Output Operator
+                        key expressions: _col0 (type: string)
+                        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
         Union 2 
@@ -1355,32 +1359,40 @@ 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)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1000 Data size: 87000 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
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1000 Data size: 87000 Basic stats: COMPLETE Column stats: COMPLETE
             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: 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)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: string)
-                      Statistics: Num rows: 1000 Data size: 87000 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
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1000 Data size: 87000 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: no inputs
         Map 4 
diff --git a/ql/src/test/results/clientpositive/llap/transitive_not_null.q.out b/ql/src/test/results/clientpositive/llap/transitive_not_null.q.out
new file mode 100644
index 0000000..a0089ac
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/transitive_not_null.q.out
@@ -0,0 +1,351 @@
+PREHOOK: query: create table tbl_1(i1 int, j1 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_1
+POSTHOOK: query: create table tbl_1(i1 int, j1 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_1
+PREHOOK: query: insert into tbl_1 values(1,2),(1,null), (null, 200), (45,68)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_1
+POSTHOOK: query: insert into tbl_1 values(1,2),(1,null), (null, 200), (45,68)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_1
+POSTHOOK: Lineage: tbl_1.i1 SCRIPT []
+POSTHOOK: Lineage: tbl_1.j1 SCRIPT []
+PREHOOK: query: create table tbl_2(i2 int, j2 int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl_2
+POSTHOOK: query: create table tbl_2(i2 int, j2 int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl_2
+PREHOOK: query: insert into tbl_2 values(1,2),(1,null), (null, 200), (45,68)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@tbl_2
+POSTHOOK: query: insert into tbl_2 values(1,2),(1,null), (null, 200), (45,68)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@tbl_2
+POSTHOOK: Lineage: tbl_2.i2 SCRIPT []
+POSTHOOK: Lineage: tbl_2.j2 SCRIPT []
+PREHOOK: query: explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveJoin(condition=[=($0, $2)], joinType=[left], algorithm=[none], cost=[not available])
+  HiveProject(i1=[$0], j1=[$1])
+    HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
+  HiveProject(i2=[$0], j2=[$1])
+    HiveFilter(condition=[IS NOT NULL($0)])
+      HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
+
+PREHOOK: query: select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+NULL	200	NULL	NULL
+1	2	1	2
+1	2	1	NULL
+1	NULL	1	2
+1	NULL	1	NULL
+45	68	45	68
+PREHOOK: query: explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveJoin(condition=[=($0, $2)], joinType=[right], algorithm=[none], cost=[not available])
+  HiveProject(i1=[$0], j1=[$1])
+    HiveFilter(condition=[IS NOT NULL($0)])
+      HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
+  HiveProject(i2=[$0], j2=[$1])
+    HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
+
+PREHOOK: query: select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+NULL	NULL	NULL	200
+1	2	1	2
+1	2	1	NULL
+1	NULL	1	2
+1	NULL	1	NULL
+45	68	45	68
+PREHOOK: query: explain cbo select * from tbl_1 full outer join tbl_2 on tbl_1.i1 = tbl_2.i2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from tbl_1 full outer join tbl_2 on tbl_1.i1 = tbl_2.i2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveJoin(condition=[=($0, $2)], joinType=[full], algorithm=[none], cost=[not available])
+  HiveProject(i1=[$0], j1=[$1])
+    HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
+  HiveProject(i2=[$0], j2=[$1])
+    HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
+
+PREHOOK: query: select * from tbl_1 full outer join tbl_2 on tbl_1.i1 = tbl_2.i2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from tbl_1 full outer join tbl_2 on tbl_1.i1 = tbl_2.i2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+NULL	NULL	NULL	200
+NULL	200	NULL	NULL
+1	2	1	2
+1	2	1	NULL
+1	NULL	1	2
+1	NULL	1	NULL
+45	68	45	68
+PREHOOK: query: explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[left], algorithm=[none], cost=[not available])
+  HiveProject(i1=[$0], j1=[$1])
+    HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
+  HiveProject(i2=[$0], j2=[$1])
+    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+      HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
+
+PREHOOK: query: select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+NULL	200	NULL	NULL
+1	NULL	NULL	NULL
+1	2	1	2
+45	68	45	68
+PREHOOK: query: explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[right], algorithm=[none], cost=[not available])
+  HiveProject(i1=[$0], j1=[$1])
+    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+      HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
+  HiveProject(i2=[$0], j2=[$1])
+    HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
+
+PREHOOK: query: select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1=tbl_2.j2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+NULL	NULL	NULL	200
+NULL	NULL	1	NULL
+1	2	1	2
+45	68	45	68
+PREHOOK: query: explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveJoin(condition=[AND(=($0, $2), >($1, $3))], joinType=[left], algorithm=[none], cost=[not available])
+  HiveProject(i1=[$0], j1=[$1])
+    HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
+  HiveProject(i2=[$0], j2=[$1])
+    HiveFilter(condition=[IS NOT NULL($0)])
+      HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
+
+PREHOOK: query: select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from tbl_1 left join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+NULL	200	NULL	NULL
+1	2	NULL	NULL
+1	NULL	NULL	NULL
+45	68	NULL	NULL
+PREHOOK: query: explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveJoin(condition=[AND(=($0, $2), >($1, $3))], joinType=[right], algorithm=[none], cost=[not available])
+  HiveProject(i1=[$0], j1=[$1])
+    HiveFilter(condition=[IS NOT NULL($0)])
+      HiveTableScan(table=[[default, tbl_1]], table:alias=[tbl_1])
+  HiveProject(i2=[$0], j2=[$1])
+    HiveTableScan(table=[[default, tbl_2]], table:alias=[tbl_2])
+
+PREHOOK: query: select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: select * from tbl_1 right join tbl_2 on tbl_1.i1 = tbl_2.i2 AND tbl_1.j1>tbl_2.j2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+NULL	NULL	NULL	200
+NULL	NULL	1	2
+NULL	NULL	1	NULL
+NULL	NULL	45	68
+PREHOOK: query: explain cbo SELECT t0.col0, t0.col1
+FROM
+  (
+    SELECT i1 as col0, j1 as col1 FROM tbl_1
+  ) AS t0
+  LEFT JOIN
+  (
+    SELECT i2 as col0, j2 as col1 FROM tbl_2
+  ) AS t1
+ON t0.col0 = t1.col0 AND t0.col1 = t1.col1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl_1
+PREHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo SELECT t0.col0, t0.col1
+FROM
+  (
+    SELECT i1 as col0, j1 as col1 FROM tbl_1
+  ) AS t0
+  LEFT JOIN
+  (
+    SELECT i2 as col0, j2 as col1 FROM tbl_2
+  ) AS t1
+ON t0.col0 = t1.col0 AND t0.col1 = t1.col1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl_1
+POSTHOOK: Input: default@tbl_2
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(col0=[$0], col1=[$1])
+  HiveJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[left], algorithm=[none], cost=[not available])
... 10197 lines suppressed ...