You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2020/08/08 01:22:54 UTC

[hive] branch master updated: HIVE-23716: Support Anti Join in Hive (Mahesh Kumar Behera, reviewed by Vineet Garg, Ramesh Kumar Thangarajan, Panagiotis Garefalakis, Jesus Camacho Rodriguez)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 14c653a  HIVE-23716: Support Anti Join in Hive (Mahesh Kumar Behera, reviewed by Vineet Garg, Ramesh Kumar Thangarajan, Panagiotis Garefalakis, Jesus Camacho Rodriguez)
14c653a is described below

commit 14c653a5fed76649b128945f266c350a9db324af
Author: mahesh kumar behera <ma...@apache.org>
AuthorDate: Sat Aug 8 06:52:37 2020 +0530

    HIVE-23716: Support Anti Join in Hive (Mahesh Kumar Behera, reviewed by Vineet Garg, Ramesh Kumar Thangarajan, Panagiotis Garefalakis, Jesus Camacho Rodriguez)
    
    Closes apache/hive#1147
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |    3 +-
 .../mapjoin/MapJoinMultiKeyBench.java              |    2 +-
 .../test/resources/testconfiguration.properties    |    1 +
 .../apache/hadoop/hive/ql/parse/FromClauseParser.g |    1 +
 .../org/apache/hadoop/hive/ql/parse/HintParser.g   |    1 +
 .../org/apache/hadoop/hive/ql/parse/HiveParser.g   |    1 +
 .../hadoop/hive/ql/exec/CommonJoinOperator.java    |   61 +-
 .../apache/hadoop/hive/ql/exec/JoinOperator.java   |    2 +-
 ...ectorMapJoinAntiJoinGenerateResultOperator.java |  227 +++
 .../mapjoin/VectorMapJoinAntiJoinLongOperator.java |  307 ++++
 .../VectorMapJoinAntiJoinMultiKeyOperator.java     |  392 +++++
 .../VectorMapJoinAntiJoinStringOperator.java       |  363 +++++
 .../ql/optimizer/ConstantPropagateProcFactory.java |    2 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java      |    1 +
 .../hadoop/hive/ql/optimizer/MapJoinProcessor.java |    3 +-
 .../hive/ql/optimizer/calcite/HiveCalciteUtil.java |   23 +
 .../ql/optimizer/calcite/HiveRelFactories.java     |    5 +
 .../HiveRelOptMaterializationValidator.java        |   10 +
 .../hive/ql/optimizer/calcite/HiveRelOptUtil.java  |    7 +
 .../calcite/HiveSubQRemoveRelBuilder.java          |   11 +-
 .../calcite/reloperators/HiveAntiJoin.java         |   97 ++
 .../calcite/rules/HiveAntiSemiJoinRule.java        |  160 ++
 .../calcite/rules/HiveJoinAddNotNullRule.java      |   16 +-
 .../calcite/rules/HiveJoinConstraintsRule.java     |    9 +-
 .../rules/HiveJoinProjectTransposeRule.java        |    8 +
 .../HiveJoinPushTransitivePredicatesRule.java      |    4 +
 .../rules/HiveProjectJoinTransposeRule.java        |    3 +-
 .../calcite/rules/HiveRelDecorrelator.java         |   58 +-
 .../calcite/rules/HiveRemoveGBYSemiJoinRule.java   |   26 +-
 .../calcite/rules/HiveSubQueryRemoveRule.java      |   30 +-
 .../calcite/stats/EstimateUniqueKeys.java          |    9 +
 .../calcite/stats/HiveRelMdDistinctRowCount.java   |    8 +
 .../calcite/stats/HiveRelMdPredicates.java         |    5 +-
 .../optimizer/calcite/stats/HiveRelMdRowCount.java |   33 +-
 .../calcite/stats/HiveRelMdSelectivity.java        |   10 +-
 .../ql/optimizer/calcite/stats/HiveRelMdSize.java  |   16 +-
 .../optimizer/calcite/translator/ASTBuilder.java   |    3 +
 .../optimizer/calcite/translator/ASTConverter.java |    2 +-
 .../calcite/translator/PlanModifierForASTConv.java |    8 +-
 .../translator/opconventer/HiveOpConverter.java    |    5 +-
 .../translator/opconventer/JoinVisitor.java        |   28 +-
 .../correlation/CorrelationOptimizer.java          |    3 +-
 .../hive/ql/optimizer/physical/Vectorizer.java     |   16 +
 .../stats/annotation/StatsRulesProcFactory.java    |   12 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |   39 +-
 .../org/apache/hadoop/hive/ql/parse/JoinType.java  |    2 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java     |   12 +-
 .../apache/hadoop/hive/ql/plan/JoinCondDesc.java   |    9 +
 .../org/apache/hadoop/hive/ql/plan/JoinDesc.java   |    1 +
 .../hadoop/hive/ql/plan/VectorMapJoinDesc.java     |    3 +-
 .../hive/ql/ppd/PredicateTransitivePropagate.java  |    1 +
 .../hadoop/hive/ql/ppd/SyntheticJoinPredicate.java |    6 +
 .../ql/exec/vector/mapjoin/MapJoinTestConfig.java  |    6 +-
 .../ql/exec/vector/mapjoin/MapJoinTestData.java    |    1 +
 .../exec/vector/mapjoin/TestMapJoinOperator.java   |    6 +-
 ql/src/test/queries/clientpositive/antijoin.q      |   67 +
 ql/src/test/queries/clientpositive/explainuser_1.q |   43 +
 .../clientpositive/external_jdbc_table_perf.q      |  193 +++
 ql/src/test/queries/clientpositive/lineage3.q      |    7 +
 .../queries/clientpositive/subquery_in_having.q    |   16 +
 .../test/queries/clientpositive/subquery_multi.q   |   42 +
 .../test/queries/clientpositive/subquery_scalar.q  |    6 +-
 .../results/clientpositive/llap/antijoin.q.out     | 1701 ++++++++++++++++++++
 .../clientpositive/llap/explainuser_1.q.out        |  418 ++++-
 .../llap/external_jdbc_table_perf.q.out            | 1116 ++++++++++++-
 .../results/clientpositive/llap/lineage3.q.out     |   15 +-
 .../clientpositive/llap/subquery_in_having.q.out   |  368 +++++
 .../clientpositive/llap/subquery_multi.q.out       | 1079 +++++++++++++
 .../clientpositive/llap/subquery_notexists.q.out   |  333 ++--
 .../llap/subquery_notexists_having.q.out           |  147 +-
 .../clientpositive/llap/subquery_scalar.q.out      |  144 ++
 .../clientpositive/perf/tez/cbo_query16.q.out      |   50 +-
 .../clientpositive/perf/tez/cbo_query69.q.out      |   83 +-
 .../clientpositive/perf/tez/cbo_query78.q.out      |   45 +-
 .../clientpositive/perf/tez/cbo_query94.q.out      |   50 +-
 .../perf/tez/constraints/cbo_query16.q.out         |   48 +-
 .../perf/tez/constraints/cbo_query69.q.out         |   85 +-
 .../perf/tez/constraints/cbo_query78.q.out         |   39 +-
 .../perf/tez/constraints/cbo_query94.q.out         |   48 +-
 .../perf/tez/constraints/query16.q.out             |  218 ++-
 .../perf/tez/constraints/query69.q.out             |  346 ++--
 .../perf/tez/constraints/query78.q.out             |  303 ++--
 .../perf/tez/constraints/query94.q.out             |  218 ++-
 .../results/clientpositive/perf/tez/query16.q.out  |  222 ++-
 .../results/clientpositive/perf/tez/query69.q.out  |  354 ++--
 .../results/clientpositive/perf/tez/query78.q.out  |  327 ++--
 .../results/clientpositive/perf/tez/query94.q.out  |  222 ++-
 87 files changed, 8530 insertions(+), 1901 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 58615e1..742dddf 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2249,7 +2249,8 @@ public class HiveConf extends Configuration {
         "Whether Hive enables the optimization about converting common join into mapjoin based on the input file size. \n" +
         "If this parameter is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than the\n" +
         "specified size, the join is directly converted to a mapjoin (there is no conditional task)."),
-
+    HIVE_CONVERT_ANTI_JOIN("hive.auto.convert.anti.join", true,
+        "Whether Hive enables the optimization about converting join with null filter to anti join"),
     HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD("hive.auto.convert.join.noconditionaltask.size",
         10000000L,
         "If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect. \n" +
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/MapJoinMultiKeyBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/MapJoinMultiKeyBench.java
index ca76e6c..d54002a 100644
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/MapJoinMultiKeyBench.java
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/MapJoinMultiKeyBench.java
@@ -36,7 +36,7 @@ import org.openjdk.jmh.runner.options.OptionsBuilder;
  * From itests/hive-jmh directory, run:
  *     java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization.mapjoin.MapJoinMultiKeyBench
  *
- *  {INNER, INNER_BIG_ONLY, LEFT_SEMI, OUTER}
+ *  {INNER, INNER_BIG_ONLY, LEFT_SEMI, OUTER, ANTI}
  *    X
  *  {ROW_MODE_HASH_MAP, ROW_MODE_OPTIMIZED, VECTOR_PASS_THROUGH, NATIVE_VECTOR_OPTIMIZED, NATIVE_VECTOR_FAST}
  *
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index e2cee9c..5d69d29 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -342,6 +342,7 @@ spark.query.files=\
   alter_merge_orc.q,\
   alter_merge_stats_orc.q,\
   annotate_stats_join.q,\
+  antijoin.q,\
   auto_join0.q,\
   auto_join1.q,\
   auto_join10.q,\
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
index e6b6fd6..b1a0b62 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
@@ -145,6 +145,7 @@ joinToken
     | KW_RIGHT (KW_OUTER)? KW_JOIN -> TOK_RIGHTOUTERJOIN
     | KW_FULL  (KW_OUTER)? KW_JOIN -> TOK_FULLOUTERJOIN
     | KW_LEFT KW_SEMI KW_JOIN      -> TOK_LEFTSEMIJOIN
+    | KW_LEFT KW_ANTI KW_JOIN -> TOK_LEFTANTISEMIJOIN
     ;
 
 lateralView
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HintParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HintParser.g
index e66b28d..f9607fc 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HintParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HintParser.g
@@ -33,6 +33,7 @@ tokens {
   TOK_HINTARGLIST;
   TOK_LEFTSEMIJOIN;
   TOK_PKFK_JOIN;
+  TOK_LEFTANTISEMIJOIN;
 }
 
 @header {
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 7a47e2d..4805323 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -304,6 +304,7 @@ TOK_USERSCRIPTCOLSCHEMA;
 TOK_RECORDREADER;
 TOK_RECORDWRITER;
 TOK_LEFTSEMIJOIN;
+TOK_LEFTANTISEMIJOIN;
 TOK_LATERAL_VIEW;
 TOK_LATERAL_VIEW_OUTER;
 TOK_TABALIAS;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
index 7a7c8a5..888aa85 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
@@ -153,6 +153,8 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
 
   transient boolean hasLeftSemiJoin = false;
 
+  transient boolean hasLeftAntiSemiJoin = false;
+
   protected transient int countAfterReport;
   protected transient int heartbeatInterval;
   protected static final int NOTSKIPBIGTABLE = -1;
@@ -365,6 +367,8 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
     for( int i = 0; i < condn.length; i++ ) {
       if(condn[i].getType() == JoinDesc.LEFT_SEMI_JOIN) {
         hasLeftSemiJoin = true;
+      } else if(condn[i].getType() == JoinDesc.ANTI_JOIN) {
+        hasLeftAntiSemiJoin = true;
       }
     }
 
@@ -509,11 +513,17 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
     }
   }
 
+  private void createForwardJoinObjectForAntiJoin(boolean[] skip) throws HiveException {
+    boolean forward = fillForwardCache(skip);
+    if (forward) {
+      internalForward(forwardCache, outputObjInspector);
+      countAfterReport = 0;
+    }
+  }
+
   // fill forwardCache with skipvector
-  // returns whether a record was forwarded
-  private boolean createForwardJoinObject(boolean[] skip) throws HiveException {
+  private boolean fillForwardCache(boolean[] skip) {
     Arrays.fill(forwardCache, null);
-
     boolean forward = false;
     for (int i = 0; i < numAliases; i++) {
       if (!skip[i]) {
@@ -523,11 +533,19 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
         forward = true;
       }
     }
+    return forward;
+  }
+
+  // returns whether a record was forwarded
+  private boolean createForwardJoinObject(boolean[] skip, boolean antiJoin) throws HiveException {
+    boolean forward = fillForwardCache(skip);
     if (forward) {
       if (needsPostEvaluation) {
         forward = !JoinUtil.isFiltered(forwardCache, residualJoinFilters, residualJoinFiltersOIs);
       }
-      if (forward) {
+
+      // For anti join, check all right side and if nothing is matched then only forward.
+      if (forward && !antiJoin) {
         // If it is not an outer join, or the post-condition filters
         // are empty or the row passed them
         internalForward(forwardCache, outputObjInspector);
@@ -620,10 +638,11 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
     boolean rightFirst = true;
     AbstractRowContainer.RowIterator<List<Object>> iter = aliasRes.rowIter();
     int pos = 0;
+
     for (List<Object> rightObj = iter.first(); !done && rightObj != null;
          rightObj = loopAgain ? rightObj : iter.next(), rightFirst = loopAgain = false, pos++) {
+      // Keep a copy of the skip vector and update the bit for current alias only in the loop.
       System.arraycopy(prevSkip, 0, skip, 0, prevSkip.length);
-
       boolean rightNull = rightObj == dummyObj[aliasNum];
       if (hasFilter(order[aliasNum])) {
         filterTags[aliasNum] = getFilterTag(rightObj);
@@ -638,6 +657,12 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
           // skipping the rest of the rows in the rhs table of the semijoin
           done = !needsPostEvaluation;
         }
+      } else if (type == JoinDesc.ANTI_JOIN) {
+        if (innerJoin(skip, left, right)) {
+          // if inner join found a match then the condition is not matched for anti join, so we can skip rest of the
+          // record. But if there is some post evaluation we have to handle that.
+          done = !needsPostEvaluation;
+        }
       } else if (type == JoinDesc.LEFT_OUTER_JOIN ||
           (type == JoinDesc.FULL_OUTER_JOIN && rightNull)) {
         int result = leftOuterJoin(skip, left, right);
@@ -668,7 +693,8 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
           needToProduceLeftRow = true;
           if (needsPostEvaluation) {
             // This is only executed for outer joins with residual filters
-            boolean forward = createForwardJoinObject(skipVectors[numAliases - 1]);
+            boolean forward = createForwardJoinObject(skipVectors[numAliases - 1],
+                                                    type == JoinDesc.ANTI_JOIN);
             producedRow |= forward;
             done = (type == JoinDesc.LEFT_SEMI_JOIN) && forward;
             if (!rightNull &&
@@ -687,7 +713,7 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
               }
             }
           } else {
-            createForwardJoinObject(skipVectors[numAliases - 1]);
+            createForwardJoinObject(skipVectors[numAliases - 1], type == JoinDesc.ANTI_JOIN);
           }
         }
       } else {
@@ -696,6 +722,17 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
       }
     }
 
+    // For anti join, we should proceed to emit records if the right side is empty or not matching.
+    if (type == JoinDesc.ANTI_JOIN && !producedRow) {
+      System.arraycopy(prevSkip, 0, skip, 0, prevSkip.length);
+      skip[right] = true;
+      if (aliasNum == numAliases - 1) {
+        createForwardJoinObjectForAntiJoin(skipVectors[numAliases - 1]);
+      } else {
+        genObject(aliasNum + 1, allLeftFirst, allLeftNull);
+      }
+    }
+
     // Consolidation for outer joins
     if (needsPostEvaluation && aliasNum == numAliases - 1 &&
             needToProduceLeftRow && !producedRow && !allLeftNull) {
@@ -935,9 +972,17 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
       for (int i = 0; i < numAliases; i++) {
         Byte alias = order[i];
         AbstractRowContainer<List<Object>> alw = storage[alias];
+        boolean isRightOfAntiJoin = (i != 0 && condn[i-1].getType() == JoinDesc.ANTI_JOIN);
 
         if (noOuterJoin) {
           if (!alw.hasRows()) {
+            if (!isRightOfAntiJoin) {
+              // For anti join the right side can be empty.
+              return;
+            }
+          } else if (isRightOfAntiJoin && !needsPostEvaluation)  {
+            // For anti join the right side should be empty. For needsPostEvaluation case we will
+            // wait till evaluation is done. For other cases we can directly return from here.
             return;
           } else if (!alw.isSingleRow()) {
             mayHasMoreThanOne = true;
@@ -968,7 +1013,7 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
 
       if (!needsPostEvaluation && !hasEmpty && !mayHasMoreThanOne) {
         genAllOneUniqueJoinObject();
-      } else if (!needsPostEvaluation && !hasEmpty && !hasLeftSemiJoin) {
+      } else if (!needsPostEvaluation && !hasEmpty && !hasLeftSemiJoin && !hasLeftAntiSemiJoin) {
         genUniqueJoinObject(0, 0);
       } else {
         genJoinObject();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
index 47ae047..306d6dd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
@@ -97,7 +97,7 @@ public class JoinOperator extends CommonJoinOperator<JoinDesc> implements Serial
       List keyObject = (List) soi.getStructFieldData(row, sf);
       // Are we consuming too much memory
       if (alias == numAliases - 1 && !(handleSkewJoin && skewJoinKeyContext.currBigKeyTag >= 0) &&
-          !hasLeftSemiJoin) {
+          !hasLeftSemiJoin && !hasLeftAntiSemiJoin) {
         if (sz == joinEmitInterval && !hasFilter(condn[alias-1].getLeft()) &&
                 !hasFilter(condn[alias-1].getRight())) {
           // The input is sorted by alias, so if we are already in the last join
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinGenerateResultOperator.java
new file mode 100644
index 0000000..962f920
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinGenerateResultOperator.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.mapjoin;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.JoinUtil;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinHashSet;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinHashSetResult;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinHashTableResult;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+// TODO : This class is duplicate of semi join. Need to do a refactoring to merge it with semi join.
+/**
+ * This class has methods for generating vectorized join results for Anti joins.
+ * The big difference between inner joins and anti joins is existence testing.
+ * Inner joins use a hash map to lookup the 1 or more small table values.
+ * Anti joins are a specialized join for outputting big table rows whose key does not
+ * exists in the small table.
+ *
+ * No small table values are needed for anti since they would be empty.  So,
+ * we use a hash set as the hash table.  Hash sets just report whether a key exists.  This
+ * is a big performance optimization.
+ */
+public abstract class VectorMapJoinAntiJoinGenerateResultOperator
+        extends VectorMapJoinGenerateResultOperator {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LoggerFactory.getLogger(VectorMapJoinAntiJoinGenerateResultOperator.class.getName());
+
+  // Anti join specific members.
+
+  // An array of hash set results so we can do lookups on the whole batch before output result
+  // generation.
+  protected transient VectorMapJoinHashSetResult hashSetResults[];
+
+  // Pre-allocated member for storing the (physical) batch index of matching row (single- or
+  // multi-small-table-valued) indexes during a process call.
+  protected transient int[] allMatchs;
+
+  // Pre-allocated member for storing the (physical) batch index of rows that need to be spilled.
+  protected transient int[] spills;
+
+  // Pre-allocated member for storing index into the hashSetResults for each spilled row.
+  protected transient int[] spillHashMapResultIndices;
+
+  /** Kryo ctor. */
+  protected VectorMapJoinAntiJoinGenerateResultOperator() {
+    super();
+  }
+
+  public VectorMapJoinAntiJoinGenerateResultOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinAntiJoinGenerateResultOperator(CompilationOpContext ctx, OperatorDesc conf,
+                                                     VectorizationContext vContext, VectorDesc vectorDesc) throws HiveException {
+    super(ctx, conf, vContext, vectorDesc);
+  }
+
+  /*
+   * Setup our anti join specific members.
+   */
+  protected void commonSetup() throws HiveException {
+    super.commonSetup();
+
+    // Anti join specific.
+    VectorMapJoinHashSet baseHashSet = (VectorMapJoinHashSet) vectorMapJoinHashTable;
+
+    hashSetResults = new VectorMapJoinHashSetResult[VectorizedRowBatch.DEFAULT_SIZE];
+    for (int i = 0; i < hashSetResults.length; i++) {
+      hashSetResults[i] = baseHashSet.createHashSetResult();
+    }
+
+    allMatchs = new int[VectorizedRowBatch.DEFAULT_SIZE];
+
+    spills = new int[VectorizedRowBatch.DEFAULT_SIZE];
+    spillHashMapResultIndices = new int[VectorizedRowBatch.DEFAULT_SIZE];
+  }
+
+  //-----------------------------------------------------------------------------------------------
+
+  /*
+   * Anti join (hash set).
+   */
+
+  /**
+   * Generate the anti join output results for one vectorized row batch. The result is modified during hash
+   * table match to reverse the result for anti join. So here matching means, the row can be emitted as the row
+   * is actually not matching.
+   *
+   * @param batch
+   *          The big table batch with any matching and any non matching rows both as
+   *          selected in use.
+   * @param allMatchCount
+   *          Number of matches in allMatches.
+   * @param spillCount
+   *          Number of spills in spills.
+   * @param hashTableResults
+   *          The array of all hash table results for the batch. We need the
+   *          VectorMapJoinHashTableResult for the spill information.
+   */
+  protected void finishAnti(VectorizedRowBatch batch,
+                            int allMatchCount, int spillCount,
+                            VectorMapJoinHashTableResult[] hashTableResults) throws HiveException, IOException {
+
+    // Get rid of spills before we start modifying the batch.
+    if (spillCount > 0) {
+      spillHashMapBatch(batch, hashTableResults,
+          spills, spillHashMapResultIndices, spillCount);
+    }
+
+    /*
+     * Optimize by running value expressions only over the matched rows.
+     */
+    if (allMatchCount > 0 && bigTableValueExpressions != null) {
+      performValueExpressions(batch, allMatchs, allMatchCount);
+    }
+
+    batch.size = generateHashSetResults(batch, allMatchs, allMatchCount);
+    batch.selectedInUse = true;
+  }
+
+  /**
+   * Generate the matching anti join output results of a vectorized row batch.
+   *
+   * @param batch
+   *          The big table batch.
+   * @param allMatches
+   *          A subset of the rows of the batch that are matches.
+   * @param allMatchCount
+   *          Number of matches in allMatches.
+   */
+  private int generateHashSetResults(VectorizedRowBatch batch,
+      int[] allMatches, int allMatchCount) {
+    int numSel = 0;
+    // Generate result within big table batch itself.
+    for (int i = 0; i < allMatchCount; i++) {
+      int batchIndex = allMatchs[i];
+      // Use the big table row as output.
+      batch.selected[numSel++] = batchIndex;
+    }
+    return numSel;
+  }
+
+  protected JoinUtil.JoinResult inverseResultForAntiJoin(JoinUtil.JoinResult joinResult) {
+    if (joinResult == JoinUtil.JoinResult.NOMATCH) {
+      return JoinUtil.JoinResult.MATCH;
+    } else if (joinResult == JoinUtil.JoinResult.MATCH) {
+      return JoinUtil.JoinResult.NOMATCH;
+    }
+    return joinResult;
+  }
+
+  /**
+   * Generate the anti join output results for one vectorized row batch with a repeated key.
+   *
+   * @param batch
+   *          The big table batch whose repeated key matches.
+   */
+  protected int generateHashSetResultRepeatedAll(VectorizedRowBatch batch) {
+    if (batch.selectedInUse) {
+      // The selected array is already filled in as we want it.
+    } else {
+      int[] selected = batch.selected;
+      for (int i = 0; i < batch.size; i++) {
+        selected[i] = i;
+      }
+      batch.selectedInUse = true;
+    }
+    return batch.size;
+  }
+
+  protected void finishAntiRepeated(VectorizedRowBatch batch, JoinUtil.JoinResult joinResult,
+                                    VectorMapJoinHashTableResult hashSetResult) throws HiveException, IOException {
+    switch (joinResult) {
+    case MATCH:
+
+      if (bigTableValueExpressions != null) {
+        // Run our value expressions over whole batch.
+        for(VectorExpression ve: bigTableValueExpressions) {
+          ve.evaluate(batch);
+        }
+      }
+
+      // Generate special repeated case.
+      batch.size = generateHashSetResultRepeatedAll(batch);
+      batch.selectedInUse = true;
+      break;
+
+    case SPILL:
+      // Whole batch is spilled.
+      spillBatchRepeated(batch, hashSetResult);
+      batch.size = 0;
+      break;
+
+    case NOMATCH:
+      // No match for entire batch.
+      batch.size = 0;
+      break;
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinLongOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinLongOperator.java
new file mode 100644
index 0000000..ea7d52d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinLongOperator.java
@@ -0,0 +1,307 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.mapjoin;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.JoinUtil;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinLongHashSet;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+// TODO : Duplicate codes need to merge with semi join.
+// Single-Column Long hash table import.
+// Single-Column Long specific imports.
+
+/*
+ * Specialized class for doing a vectorized map join that is an anti join on a Single-Column Long
+ * using a hash set.
+ */
+public class VectorMapJoinAntiJoinLongOperator extends VectorMapJoinAntiJoinGenerateResultOperator {
+
+  private static final long serialVersionUID = 1L;
+  private static final String CLASS_NAME = VectorMapJoinAntiJoinLongOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  // The above members are initialized by the constructor and must not be
+  // transient.
+
+  // The hash map for this specialized class.
+  private transient VectorMapJoinLongHashSet hashSet;
+
+  // Single-Column Long specific members.
+  // For integers, we have optional min/max filtering.
+  private transient boolean useMinMax;
+  private transient long min;
+  private transient long max;
+
+  // The column number for this one column join specialization.
+  private transient int singleJoinColumn;
+
+  // Pass-thru constructors.
+  /** Kryo ctor. */
+  protected VectorMapJoinAntiJoinLongOperator() {
+    super();
+  }
+
+  public VectorMapJoinAntiJoinLongOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinAntiJoinLongOperator(CompilationOpContext ctx, OperatorDesc conf,
+                                           VectorizationContext vContext, VectorDesc vectorDesc) throws HiveException {
+    super(ctx, conf, vContext, vectorDesc);
+  }
+
+  // Process Single-Column Long Anti Join on a vectorized row batch.
+  @Override
+  protected void commonSetup() throws HiveException {
+    super.commonSetup();
+
+    // Initialize Single-Column Long members for this specialized class.
+    singleJoinColumn = bigTableKeyColumnMap[0];
+  }
+
+  @Override
+  public void hashTableSetup() throws HiveException {
+    super.hashTableSetup();
+
+    // Get our Single-Column Long hash set information for this specialized class.
+    hashSet = (VectorMapJoinLongHashSet) vectorMapJoinHashTable;
+    useMinMax = hashSet.useMinMax();
+    if (useMinMax) {
+      min = hashSet.min();
+      max = hashSet.max();
+    }
+  }
+
+  @Override
+  public void processBatch(VectorizedRowBatch batch) throws HiveException {
+
+    try {
+      // (Currently none)
+      // antiPerBatchSetup(batch);
+
+      // For anti joins, we may apply the filter(s) now.
+      for(VectorExpression ve : bigTableFilterExpressions) {
+        ve.evaluate(batch);
+      }
+
+      final int inputLogicalSize = batch.size;
+      if (inputLogicalSize == 0) {
+        return;
+      }
+
+      // Perform any key expressions.  Results will go into scratch columns.
+      if (bigTableKeyExpressions != null) {
+        for (VectorExpression ve : bigTableKeyExpressions) {
+          ve.evaluate(batch);
+        }
+      }
+
+      // The one join column for this specialized class.
+      LongColumnVector joinColVector = (LongColumnVector) batch.cols[singleJoinColumn];
+      long[] vector = joinColVector.vector;
+
+      // Check single column for repeating.
+      boolean allKeyInputColumnsRepeating = joinColVector.isRepeating;
+
+      if (allKeyInputColumnsRepeating) {
+        // All key input columns are repeating.  Generate key once.  Lookup once.
+        // Since the key is repeated, we must use entry 0 regardless of selectedInUse.
+        JoinUtil.JoinResult joinResult;
+        if (!joinColVector.noNulls && joinColVector.isNull[0]) {
+          // For anti join, if the right side is null then its a match.
+          joinResult = JoinUtil.JoinResult.MATCH;
+        } else {
+          long key = vector[0];
+          if (useMinMax && (key < min || key > max)) {
+            // Out of range for whole batch. Its a match for anti join. We can emit the row.
+            joinResult = JoinUtil.JoinResult.MATCH;
+          } else {
+            joinResult = hashSet.contains(key, hashSetResults[0]);
+            // reverse the join result for anti join.
+            joinResult = inverseResultForAntiJoin(joinResult);
+          }
+        }
+
+        // Common repeated join result processing.
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME + " batch #" + batchCounter + " repeated joinResult " + joinResult.name());
+        }
+        finishAntiRepeated(batch, joinResult, hashSetResults[0]);
+      } else {
+        // NOT Repeating.
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME + " batch #" + batchCounter + " non-repeated");
+        }
+
+        // We remember any matching rows in matches / matchSize.  At the end of the loop,
+        // selected / batch.size will represent both matching and non-matching rows for outer join.
+        // Only deferred rows will have been removed from selected.
+        int selected[] = batch.selected;
+        boolean selectedInUse = batch.selectedInUse;
+
+        int hashSetResultCount = 0;
+        int allMatchCount = 0;
+        int spillCount = 0;
+        long saveKey = 0;
+
+        // We optimize performance by only looking up the first key in a series of equal keys.
+        boolean haveSaveKey = false;
+        JoinUtil.JoinResult saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+
+        // Logical loop over the rows in the batch since the batch may have selected in use.
+        for (int logical = 0; logical < inputLogicalSize; logical++) {
+          int batchIndex = (selectedInUse ? selected[logical] : logical);
+
+          // Single-Column Long get key.
+          long currentKey;
+          boolean isNull;
+          if (!joinColVector.noNulls && joinColVector.isNull[batchIndex]) {
+            currentKey = 0;
+            isNull = true;
+          } else {
+            currentKey = vector[batchIndex];
+            isNull = false;
+          }
+
+          // Equal key series checking.
+          if (isNull || !haveSaveKey || currentKey != saveKey) {
+            // New key.
+            if (haveSaveKey) {
+              // Move on with our counts.
+              switch (saveJoinResult) {
+              case MATCH:
+                // We have extracted the existence from the hash set result, so we don't keep it.
+                break;
+              case SPILL:
+                // We keep the hash set result for its spill information.
+                hashSetResultCount++;
+                break;
+              case NOMATCH:
+                break;
+              }
+            }
+
+            if (isNull) {
+              saveJoinResult = JoinUtil.JoinResult.MATCH;
+              haveSaveKey = false;
+            } else {
+              // Regardless of our matching result, we keep that information to make multiple use
+              // of it for a possible series of equal keys.
+              haveSaveKey = true;
+              saveKey = currentKey;
+              if (useMinMax && (currentKey < min || currentKey > max)) {
+                saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+              } else {
+                saveJoinResult = hashSet.contains(currentKey, hashSetResults[hashSetResultCount]);
+              }
+
+              // Reverse the match result for anti join.
+              // Key out of range for whole hash table, is a valid match for anti join.
+              saveJoinResult = inverseResultForAntiJoin(saveJoinResult);
+            }
+
+            // Common anti join result processing.
+            switch (saveJoinResult) {
+            case MATCH:
+              allMatchs[allMatchCount++] = batchIndex;
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH isSingleValue " + equalKeySeriesIsSingleValue[equalKeySeriesCount] + " currentKey " + currentKey);
+              break;
+
+            case SPILL:
+              spills[spillCount] = batchIndex;
+              spillHashMapResultIndices[spillCount] = hashSetResultCount;
+              spillCount++;
+              break;
+
+            case NOMATCH:
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH" + " currentKey " + currentKey);
+              break;
+            }
+          } else {
+            // Series of equal keys.
+            switch (saveJoinResult) {
+            case MATCH:
+              allMatchs[allMatchCount++] = batchIndex;
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH duplicate");
+              break;
+
+            case SPILL:
+              spills[spillCount] = batchIndex;
+              spillHashMapResultIndices[spillCount] = hashSetResultCount;
+              spillCount++;
+              break;
+
+            case NOMATCH:
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH duplicate");
+              break;
+            }
+          }
+        }
+
+        if (haveSaveKey) {
+          // Update our counts for the last key.
+          switch (saveJoinResult) {
+          case MATCH:
+            // We have extracted the existence from the hash set result, so we don't keep it.
+            break;
+          case SPILL:
+            // We keep the hash set result for its spill information.
+            hashSetResultCount++;
+            break;
+          case NOMATCH:
+            break;
+          }
+        }
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME +
+              " allMatchs " + intArrayToRangesString(allMatchs, allMatchCount) +
+              " spills " + intArrayToRangesString(spills, spillCount) +
+              " spillHashMapResultIndices " + intArrayToRangesString(spillHashMapResultIndices, spillCount) +
+              " hashMapResults " + Arrays.toString(Arrays.copyOfRange(hashSetResults, 0, hashSetResultCount)));
+        }
+
+        finishAnti(batch, allMatchCount, spillCount, hashSetResults);
+      }
+
+      if (batch.size > 0) {
+        // Forward any remaining selected rows.
+        forwardBigTableBatch(batch);
+      }
+
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinMultiKeyOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinMultiKeyOperator.java
new file mode 100644
index 0000000..cbccb23
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinMultiKeyOperator.java
@@ -0,0 +1,392 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.mapjoin;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.JoinUtil;
+import org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinBytesHashSet;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorDesc;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+// Multi-Key hash table import.
+// Multi-Key specific imports.
+
+// TODO : Duplicate codes need to merge with semi join.
+/*
+ * Specialized class for doing a vectorized map join that is an anti join on Multi-Key
+ * using hash set.
+ */
+public class VectorMapJoinAntiJoinMultiKeyOperator extends VectorMapJoinAntiJoinGenerateResultOperator {
+
+  private static final long serialVersionUID = 1L;
+
+  //------------------------------------------------------------------------------------------------
+
+  private static final String CLASS_NAME = VectorMapJoinAntiJoinMultiKeyOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  // (none)
+
+  // The above members are initialized by the constructor and must not be
+  // transient.
+  //---------------------------------------------------------------------------
+
+  // The hash map for this specialized class.
+  private transient VectorMapJoinBytesHashSet hashSet;
+
+  //---------------------------------------------------------------------------
+  // Multi-Key specific members.
+  //
+
+  // Object that can take a set of columns in row in a vectorized row batch and serialized it.
+  // Known to not have any nulls.
+  private transient VectorSerializeRow keyVectorSerializeWrite;
+
+  // The BinarySortable serialization of the current key.
+  private transient Output currentKeyOutput;
+
+  // The BinarySortable serialization of the saved key for a possible series of equal keys.
+  private transient Output saveKeyOutput;
+
+  //---------------------------------------------------------------------------
+  // Pass-thru constructors.
+  //
+
+  /** Kryo ctor. */
+  protected VectorMapJoinAntiJoinMultiKeyOperator() {
+    super();
+  }
+
+  public VectorMapJoinAntiJoinMultiKeyOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinAntiJoinMultiKeyOperator(CompilationOpContext ctx, OperatorDesc conf,
+                                               VectorizationContext vContext, VectorDesc vectorDesc) throws HiveException {
+    super(ctx, conf, vContext, vectorDesc);
+  }
+
+  //---------------------------------------------------------------------------
+  // Process Multi-Key Anti Join on a vectorized row batch.
+  //
+
+  @Override
+  protected void commonSetup() throws HiveException {
+    super.commonSetup();
+
+    /*
+     * Initialize Multi-Key members for this specialized class.
+     */
+
+    keyVectorSerializeWrite = new VectorSerializeRow(BinarySortableSerializeWrite.with(
+            this.getConf().getKeyTblDesc().getProperties(), bigTableKeyColumnMap.length));
+    keyVectorSerializeWrite.init(bigTableKeyTypeInfos, bigTableKeyColumnMap);
+
+    currentKeyOutput = new Output();
+    saveKeyOutput = new Output();
+  }
+
+  @Override
+  public void hashTableSetup() throws HiveException {
+    super.hashTableSetup();
+
+    /*
+     * Get our Multi-Key hash set information for this specialized class.
+     */
+
+    hashSet = (VectorMapJoinBytesHashSet) vectorMapJoinHashTable;
+  }
+
+  @Override
+  public void processBatch(VectorizedRowBatch batch) throws HiveException {
+
+    try {
+
+      // Do the per-batch setup for an anti join.
+
+      // (Currently none)
+      // antiPerBatchSetup(batch);
+
+      // For anti joins, we may apply the filter(s) now.
+      for(VectorExpression ve : bigTableFilterExpressions) {
+        ve.evaluate(batch);
+      }
+
+      final int inputLogicalSize = batch.size;
+      if (inputLogicalSize == 0) {
+        return;
+      }
+
+      // Perform any key expressions.  Results will go into scratch columns.
+      if (bigTableKeyExpressions != null) {
+        for (VectorExpression ve : bigTableKeyExpressions) {
+          ve.evaluate(batch);
+        }
+      }
+
+      /*
+       * Multi-Key specific declarations.
+       */
+
+      // None.
+
+      /*
+       * Multi-Key Long check for repeating.
+       */
+
+      // If all BigTable input columns to key expressions are isRepeating, then
+      // calculate key once; lookup once.
+      boolean allKeyInputColumnsRepeating;
+      if (bigTableKeyColumnMap.length == 0) {
+       allKeyInputColumnsRepeating = false;
+      } else {
+        allKeyInputColumnsRepeating = true;
+        for (int i = 0; i < bigTableKeyColumnMap.length; i++) {
+          if (!batch.cols[bigTableKeyColumnMap[i]].isRepeating) {
+            allKeyInputColumnsRepeating =  false;
+            break;
+          }
+        }
+      }
+
+      if (allKeyInputColumnsRepeating) {
+
+        /*
+         * Repeating.
+         */
+
+        // All key input columns are repeating.  Generate key once.  Lookup once.
+        // Since the key is repeated, we must use entry 0 regardless of selectedInUse.
+
+        /*
+         * Multi-Key specific repeated lookup.
+         */
+
+        keyVectorSerializeWrite.setOutput(currentKeyOutput);
+        keyVectorSerializeWrite.serializeWrite(batch, 0);
+        JoinUtil.JoinResult joinResult;
+        if (keyVectorSerializeWrite.getHasAnyNulls()) {
+          // If right side is null, its a match for anti join.
+          joinResult = JoinUtil.JoinResult.MATCH;
+        } else {
+          byte[] keyBytes = currentKeyOutput.getData();
+          int keyLength = currentKeyOutput.getLength();
+          // LOG.debug(CLASS_NAME + " processOp all " + displayBytes(keyBytes, 0, keyLength));
+          joinResult = hashSet.contains(keyBytes, 0, keyLength, hashSetResults[0]);
+          // reverse the join result from hash table for anti join.
+          joinResult = inverseResultForAntiJoin(joinResult);
+        }
+
+        /*
+         * Common repeated join result processing.
+         */
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME + " batch #" + batchCounter + " repeated joinResult " + joinResult.name());
+        }
+        finishAntiRepeated(batch, joinResult, hashSetResults[0]);
+      } else {
+
+        /*
+         * NOT Repeating.
+         */
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME + " batch #" + batchCounter + " non-repeated");
+        }
+
+        // We remember any matching rows in matches / matchSize.  At the end of the loop,
+        // selected / batch.size will represent both matching and non-matching rows for outer join.
+        // Only deferred rows will have been removed from selected.
+        int selected[] = batch.selected;
+        boolean selectedInUse = batch.selectedInUse;
+
+        int hashSetResultCount = 0;
+        int allMatchCount = 0;
+        int spillCount = 0;
+
+        /*
+         * Multi-Key specific variables.
+         */
+
+        Output temp;
+
+        // We optimize performance by only looking up the first key in a series of equal keys.
+        boolean haveSaveKey = false;
+        JoinUtil.JoinResult saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+
+        // Logical loop over the rows in the batch since the batch may have selected in use.
+        for (int logical = 0; logical < inputLogicalSize; logical++) {
+          int batchIndex = (selectedInUse ? selected[logical] : logical);
+
+          /*
+           * Multi-Key get key.
+           */
+
+          // Generate binary sortable key for current row in vectorized row batch.
+          keyVectorSerializeWrite.setOutput(currentKeyOutput);
+          keyVectorSerializeWrite.serializeWrite(batch, batchIndex);
+          boolean isAnyNull = keyVectorSerializeWrite.getHasAnyNulls();
+
+          // LOG.debug(CLASS_NAME + " currentKey " +
+          //      VectorizedBatchUtil.displayBytes(currentKeyOutput.getData(), 0, currentKeyOutput.getLength()));
+
+          /*
+           * Equal key series checking.
+           */
+
+          if (isAnyNull || !haveSaveKey || !saveKeyOutput.arraysEquals(currentKeyOutput)) {
+
+            // New key.
+
+            if (haveSaveKey) {
+              // Move on with our counts.
+              switch (saveJoinResult) {
+              case MATCH:
+                // We have extracted the existence from the hash set result, so we don't keep it.
+                break;
+              case SPILL:
+                // We keep the hash set result for its spill information.
+                hashSetResultCount++;
+                break;
+              case NOMATCH:
+                break;
+              }
+            }
+
+            if (isAnyNull) {
+              saveJoinResult = JoinUtil.JoinResult.MATCH;
+              haveSaveKey = false;
+            } else {
+              // Regardless of our matching result, we keep that information to make multiple use
+              // of it for a possible series of equal keys.
+              haveSaveKey = true;
+  
+              /*
+               * Multi-Key specific save key and lookup.
+               */
+  
+              temp = saveKeyOutput;
+              saveKeyOutput = currentKeyOutput;
+              currentKeyOutput = temp;
+  
+              /*
+               * Multi-key specific lookup key.
+               */
+  
+              byte[] keyBytes = saveKeyOutput.getData();
+              int keyLength = saveKeyOutput.getLength();
+              saveJoinResult = hashSet.contains(keyBytes, 0, keyLength, hashSetResults[hashSetResultCount]);
+              saveJoinResult = inverseResultForAntiJoin(saveJoinResult);
+            }
+
+            /*
+             * Common anti join result processing.
+             */
+
+            switch (saveJoinResult) {
+            case MATCH:
+              allMatchs[allMatchCount++] = batchIndex;
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH isSingleValue " + equalKeySeriesIsSingleValue[equalKeySeriesCount] + " currentKey " + currentKey);
+              break;
+
+            case SPILL:
+              spills[spillCount] = batchIndex;
+              spillHashMapResultIndices[spillCount] = hashSetResultCount;
+              spillCount++;
+              break;
+
+            case NOMATCH:
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH" + " currentKey " + currentKey);
+              break;
+            }
+          } else {
+            // Series of equal keys.
+
+            switch (saveJoinResult) {
+            case MATCH:
+              allMatchs[allMatchCount++] = batchIndex;
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH duplicate");
+              break;
+
+            case SPILL:
+              spills[spillCount] = batchIndex;
+              spillHashMapResultIndices[spillCount] = hashSetResultCount;
+              spillCount++;
+              break;
+
+            case NOMATCH:
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH duplicate");
+              break;
+            }
+          }
+        }
+
+        if (haveSaveKey) {
+          // Update our counts for the last key.
+          switch (saveJoinResult) {
+          case MATCH:
+            // We have extracted the existence from the hash set result, so we don't keep it.
+            break;
+          case SPILL:
+            // We keep the hash set result for its spill information.
+            hashSetResultCount++;
+            break;
+          case NOMATCH:
+            break;
+          }
+        }
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME +
+              " allMatchs " + intArrayToRangesString(allMatchs, allMatchCount) +
+              " spills " + intArrayToRangesString(spills, spillCount) +
+              " spillHashMapResultIndices " + intArrayToRangesString(spillHashMapResultIndices, spillCount) +
+              " hashMapResults " + Arrays.toString(Arrays.copyOfRange(hashSetResults, 0, hashSetResultCount)));
+        }
+
+        finishAnti(batch, allMatchCount, spillCount, hashSetResults);
+      }
+
+      if (batch.size > 0) {
+        // Forward any remaining selected rows.
+        forwardBigTableBatch(batch);
+      }
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinStringOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinStringOperator.java
new file mode 100644
index 0000000..4e6f247
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinStringOperator.java
@@ -0,0 +1,363 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.mapjoin;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.JoinUtil;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinBytesHashSet;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+// Single-Column String hash table import.
+// Single-Column String specific imports.
+
+// TODO : Duplicate codes need to merge with semi join.
+/*
+ * Specialized class for doing a vectorized map join that is an anti join on a Single-Column String
+ * using a hash set.
+ */
+public class VectorMapJoinAntiJoinStringOperator extends VectorMapJoinAntiJoinGenerateResultOperator {
+
+  private static final long serialVersionUID = 1L;
+
+  //------------------------------------------------------------------------------------------------
+
+  private static final String CLASS_NAME = VectorMapJoinAntiJoinStringOperator.class.getName();
+  private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+  protected String getLoggingPrefix() {
+    return super.getLoggingPrefix(CLASS_NAME);
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  // (none)
+
+  // The above members are initialized by the constructor and must not be
+  // transient.
+  //---------------------------------------------------------------------------
+
+  // The hash map for this specialized class.
+  private transient VectorMapJoinBytesHashSet hashSet;
+
+  //---------------------------------------------------------------------------
+  // Single-Column String specific members.
+  //
+
+  // The column number for this one column join specialization.
+  private transient int singleJoinColumn;
+
+  //---------------------------------------------------------------------------
+  // Pass-thru constructors.
+  //
+
+  /** Kryo ctor. */
+  protected VectorMapJoinAntiJoinStringOperator() {
+    super();
+  }
+
+  public VectorMapJoinAntiJoinStringOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinAntiJoinStringOperator(CompilationOpContext ctx, OperatorDesc conf,
+                                             VectorizationContext vContext, VectorDesc vectorDesc) throws HiveException {
+    super(ctx, conf, vContext, vectorDesc);
+  }
+
+  //---------------------------------------------------------------------------
+  // Process Single-Column String anti Join on a vectorized row batch.
+  //
+
+  @Override
+  protected void commonSetup() throws HiveException {
+    super.commonSetup();
+
+    /*
+     * Initialize Single-Column String members for this specialized class.
+     */
+
+    singleJoinColumn = bigTableKeyColumnMap[0];
+  }
+
+  @Override
+  public void hashTableSetup() throws HiveException {
+    super.hashTableSetup();
+
+    /*
+     * Get our Single-Column String hash set information for this specialized class.
+     */
+
+    hashSet = (VectorMapJoinBytesHashSet) vectorMapJoinHashTable;
+  }
+
+  @Override
+  public void processBatch(VectorizedRowBatch batch) throws HiveException {
+
+    try {
+
+      // Do the per-batch setup for an anti join.
+
+      // (Currently none)
+      // antiPerBatchSetup(batch);
+
+      // For anti joins, we may apply the filter(s) now.
+      for(VectorExpression ve : bigTableFilterExpressions) {
+        ve.evaluate(batch);
+      }
+
+      final int inputLogicalSize = batch.size;
+      if (inputLogicalSize == 0) {
+        return;
+      }
+
+      // Perform any key expressions.  Results will go into scratch columns.
+      if (bigTableKeyExpressions != null) {
+        for (VectorExpression ve : bigTableKeyExpressions) {
+          ve.evaluate(batch);
+        }
+      }
+
+      /*
+       * Single-Column String specific declarations.
+       */
+
+      // The one join column for this specialized class.
+      BytesColumnVector joinColVector = (BytesColumnVector) batch.cols[singleJoinColumn];
+      byte[][] vector = joinColVector.vector;
+      int[] start = joinColVector.start;
+      int[] length = joinColVector.length;
+
+      /*
+       * Single-Column Long check for repeating.
+       */
+
+      // Check single column for repeating.
+      boolean allKeyInputColumnsRepeating = joinColVector.isRepeating;
+
+      if (allKeyInputColumnsRepeating) {
+
+        /*
+         * Repeating.
+         */
+
+        // All key input columns are repeating.  Generate key once.  Lookup once.
+        // Since the key is repeated, we must use entry 0 regardless of selectedInUse.
+
+        /*
+         * Single-Column String specific repeated lookup.
+         */
+
+        JoinUtil.JoinResult joinResult;
+        if (!joinColVector.noNulls && joinColVector.isNull[0]) {
+          joinResult = JoinUtil.JoinResult.MATCH;
+        } else {
+          byte[] keyBytes = vector[0];
+          int keyStart = start[0];
+          int keyLength = length[0];
+          joinResult = hashSet.contains(keyBytes, keyStart, keyLength, hashSetResults[0]);
+          joinResult = inverseResultForAntiJoin(joinResult);
+        }
+
+        /*
+         * Common repeated join result processing.
+         */
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME + " batch #" + batchCounter + " repeated joinResult " + joinResult.name());
+        }
+        finishAntiRepeated(batch, joinResult, hashSetResults[0]);
+      } else {
+
+        /*
+         * NOT Repeating.
+         */
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME + " batch #" + batchCounter + " non-repeated");
+        }
+
+        // We remember any matching rows in matchs / matchSize.  At the end of the loop,
+        // selected / batch.size will represent both matching and non-matching rows for outer join.
+        // Only deferred rows will have been removed from selected.
+        int selected[] = batch.selected;
+        boolean selectedInUse = batch.selectedInUse;
+
+        int hashSetResultCount = 0;
+        int allMatchCount = 0;
+        int spillCount = 0;
+
+        /*
+         * Single-Column String specific variables.
+         */
+
+        int saveKeyBatchIndex = -1;
+
+        // We optimize performance by only looking up the first key in a series of equal keys.
+        boolean haveSaveKey = false;
+        JoinUtil.JoinResult saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+
+        // Logical loop over the rows in the batch since the batch may have selected in use.
+        for (int logical = 0; logical < inputLogicalSize; logical++) {
+          int batchIndex = (selectedInUse ? selected[logical] : logical);
+
+          /*
+           * Single-Column String get key.
+           */
+
+          // Implicit -- use batchIndex.
+          boolean isNull = !joinColVector.noNulls && joinColVector.isNull[batchIndex];
+
+          /*
+           * Equal key series checking.
+           */
+
+          if (isNull || !haveSaveKey ||
+              !StringExpr.equal(vector[saveKeyBatchIndex], start[saveKeyBatchIndex], length[saveKeyBatchIndex],
+                      vector[batchIndex], start[batchIndex], length[batchIndex])) {
+
+            // New key.
+
+            if (haveSaveKey) {
+              // Move on with our counts.
+              switch (saveJoinResult) {
+              case MATCH:
+                // We have extracted the existence from the hash set result, so we don't keep it.
+                break;
+              case SPILL:
+                // We keep the hash set result for its spill information.
+                hashSetResultCount++;
+                break;
+              case NOMATCH:
+                break;
+              }
+            }
+
+            if (isNull) {
+              saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+              haveSaveKey = false;
+            } else {
+              // Regardless of our matching result, we keep that information to make multiple use
+              // of it for a possible series of equal keys.
+              haveSaveKey = true;
+  
+              /*
+               * Single-Column String specific save key and lookup.
+               */
+  
+              saveKeyBatchIndex = batchIndex;
+  
+              /*
+               * Single-Column String specific lookup key.
+               */
+  
+              byte[] keyBytes = vector[batchIndex];
+              int keyStart = start[batchIndex];
+              int keyLength = length[batchIndex];
+              saveJoinResult = hashSet.contains(keyBytes, keyStart, keyLength, hashSetResults[hashSetResultCount]);
+              saveJoinResult = inverseResultForAntiJoin(saveJoinResult);
+            }
+
+            /*
+             * Common anti join result processing.
+             */
+
+            switch (saveJoinResult) {
+            case MATCH:
+              allMatchs[allMatchCount++] = batchIndex;
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH isSingleValue " + equalKeySeriesIsSingleValue[equalKeySeriesCount] + " currentKey " + currentKey);
+              break;
+
+            case SPILL:
+              spills[spillCount] = batchIndex;
+              spillHashMapResultIndices[spillCount] = hashSetResultCount;
+              spillCount++;
+              break;
+
+            case NOMATCH:
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH" + " currentKey " + currentKey);
+              break;
+            }
+          } else {
+            // Series of equal keys.
+
+            switch (saveJoinResult) {
+            case MATCH:
+              allMatchs[allMatchCount++] = batchIndex;
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH duplicate");
+              break;
+
+            case SPILL:
+              spills[spillCount] = batchIndex;
+              spillHashMapResultIndices[spillCount] = hashSetResultCount;
+              spillCount++;
+              break;
+
+            case NOMATCH:
+              // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH duplicate");
+              break;
+            }
+          }
+        }
+
+        if (haveSaveKey) {
+          // Update our counts for the last key.
+          switch (saveJoinResult) {
+          case MATCH:
+            // We have extracted the existence from the hash set result, so we don't keep it.
+            break;
+          case SPILL:
+            // We keep the hash set result for its spill information.
+            hashSetResultCount++;
+            break;
+          case NOMATCH:
+            break;
+          }
+        }
+
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(CLASS_NAME +
+              " allMatchs " + intArrayToRangesString(allMatchs, allMatchCount) +
+              " spills " + intArrayToRangesString(spills, spillCount) +
+              " spillHashMapResultIndices " + intArrayToRangesString(spillHashMapResultIndices, spillCount) +
+              " hashMapResults " + Arrays.toString(Arrays.copyOfRange(hashSetResults, 0, hashSetResultCount)));
+        }
+
+        finishAnti(batch, allMatchCount, spillCount, hashSetResults);
+      }
+
+      if (batch.size > 0) {
+        // Forward any remaining selected rows.
+        forwardBigTableBatch(batch);
+      }
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index 2d3daeb..f43ba92 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -1418,7 +1418,7 @@ public final class ConstantPropagateProcFactory {
     private boolean skipFolding(JoinDesc joinDesc) {
       for (JoinCondDesc cond : joinDesc.getConds()) {
         if (cond.getType() == JoinDesc.INNER_JOIN || cond.getType() == JoinDesc.UNIQUE_JOIN
-            || cond.getType() == JoinDesc.LEFT_SEMI_JOIN) {
+            || cond.getType() == JoinDesc.LEFT_SEMI_JOIN || cond.getType() == JoinDesc.ANTI_JOIN) {
           continue;
         }
         return true;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index 655b5f1..90b23e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -954,6 +954,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
       switch (joinCondDesc.getType()) {
         case JoinDesc.INNER_JOIN:
         case JoinDesc.LEFT_SEMI_JOIN:
+        case JoinDesc.ANTI_JOIN:
         case JoinDesc.UNIQUE_JOIN:
           hasOuter = false;
           break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
index 4f1c9b2..c4cb839 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
@@ -752,7 +752,8 @@ public class MapJoinProcessor extends Transform {
       seenPostitions.add(condn.getRight());
 
       if (joinType == JoinDesc.LEFT_OUTER_JOIN
-          || joinType == JoinDesc.LEFT_SEMI_JOIN) {
+          || joinType == JoinDesc.LEFT_SEMI_JOIN
+          || joinType == JoinDesc.ANTI_JOIN) {
         seenOuterJoin = true;
         if(bigTableCandidates.size() == 0) {
           bigTableCandidates.add(condn.getLeft());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
index 88aaedd..d235e3b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
@@ -1233,4 +1233,27 @@ public class HiveCalciteUtil {
     }
   }
 
+  /**
+   * Checks if any of the expression given as list expressions are from right side of the join.
+   *  This is used during anti join conversion.
+   *
+   * @param joinRel Join node whose right side has to be searched.
+   * @param expressions The list of expression to search.
+   * @return true if any of the expressions is from right side of join.
+   */
+  public static boolean hasAnyExpressionFromRightSide(RelNode joinRel, List<RexNode> expressions)  {
+    List<RelDataTypeField> joinFields = joinRel.getRowType().getFieldList();
+    int nTotalFields = joinFields.size();
+    List<RelDataTypeField> leftFields = (joinRel.getInputs().get(0)).getRowType().getFieldList();
+    int nFieldsLeft = leftFields.size();
+    ImmutableBitSet rightBitmap = ImmutableBitSet.range(nFieldsLeft, nTotalFields);
+
+    for (RexNode node : expressions) {
+      ImmutableBitSet inputBits = RelOptUtil.InputFinder.bits(node);
+      if (rightBitmap.contains(inputBits)) {
+        return true;
+      }
+    }
+    return false;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
index 6be826e..312de0f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
@@ -46,6 +46,7 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExcept;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIntersect;
@@ -170,6 +171,10 @@ public class HiveRelFactories {
         final RelOptCluster cluster = left.getCluster();
         return HiveSemiJoin.getSemiJoin(cluster, left.getTraitSet(), left, right, condition);
       }
+      if (joinType == JoinRelType.ANTI) {
+        final RelOptCluster cluster = left.getCluster();
+        return HiveAntiJoin.getAntiJoin(cluster, left.getTraitSet(), left, right, condition);
+      }
       return HiveJoin.getJoin(left.getCluster(), left, right, condition, joinType);
     }
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
index 1aa1731..90ece35 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
@@ -40,6 +40,7 @@ import org.apache.calcite.util.Util;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExcept;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIntersect;
@@ -138,6 +139,8 @@ public class HiveRelOptMaterializationValidator extends HiveRelShuttleImpl {
       return visit((HiveSemiJoin) node);
     } else if (node instanceof HiveExcept) {
       return visit((HiveExcept) node);
+    } else if (node instanceof HiveAntiJoin) {
+      return visit((HiveAntiJoin) node);
     } else if (node instanceof HiveIntersect) {
       return visit((HiveIntersect) node);
     }
@@ -253,6 +256,13 @@ public class HiveRelOptMaterializationValidator extends HiveRelShuttleImpl {
     return visitChildren(semiJoin);
   }
 
+  private RelNode visit(HiveAntiJoin antiJoin) {
+    setAutomaticRewritingInvalidReason("Statement has unsupported join type: anti join.");
+    checkExpr(antiJoin.getCondition());
+    checkExpr(antiJoin.getJoinFilter());
+    return visitChildren(antiJoin);
+  }
+
   // Note: Not currently part of the HiveRelNode interface
   private RelNode visit(HiveExcept except) {
     setAutomaticRewritingInvalidReason("Statement has unsupported operator: except.");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
index 1e2c1e2..9d45c6c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
@@ -747,6 +747,12 @@ public class HiveRelOptUtil extends RelOptUtil {
     final RelNode nonFkInput = leftInputPotentialFK ? join.getRight() : join.getLeft();
     final RewritablePKFKJoinInfo nonRewritable = RewritablePKFKJoinInfo.of(false, null);
 
+    // TODO : Need to handle Anti join.
+    // https://issues.apache.org/jira/browse/HIVE-23906
+    if (joinType == JoinRelType.ANTI) {
+      return nonRewritable;
+    }
+
     if (joinType != JoinRelType.INNER && !join.isSemiJoin()) {
       // If it is not an inner, we transform it as the metadata
       // providers for expressions do not pull information through
@@ -854,6 +860,7 @@ public class HiveRelOptUtil extends RelOptUtil {
             if (ecT.getEquivalenceClassesMap().containsKey(uniqueKeyColumnRef) &&
                 ecT.getEquivalenceClassesMap().get(uniqueKeyColumnRef).contains(foreignKeyColumnRef)) {
               if (foreignKeyColumnType.isNullable()) {
+                //TODO : Handle Anti Join. https://issues.apache.org/jira/browse/HIVE-23906
                 if (joinType == JoinRelType.INNER || join.isSemiJoin()) {
                   // If it is nullable and it is an INNER, we just need a IS NOT NULL filter
                   RexNode originalCondOp = refToRex.get(foreignKeyColumnRef);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveSubQRemoveRelBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveSubQRemoveRelBuilder.java
index a1d617b..f52ab60 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveSubQRemoveRelBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveSubQRemoveRelBuilder.java
@@ -1112,7 +1112,8 @@ public class HiveSubQRemoveRelBuilder {
   }
 
   public HiveSubQRemoveRelBuilder join(JoinRelType joinType, RexNode condition,
-                                       Set<CorrelationId> variablesSet, boolean createSemiJoin) {
+                                       Set<CorrelationId> variablesSet, JoinRelType semiJoinType) {
+    assert semiJoinType == JoinRelType.SEMI || semiJoinType == JoinRelType.ANTI || semiJoinType == null;
     Frame right = stack.pop();
     final Frame left = stack.pop();
     final RelNode join;
@@ -1138,12 +1139,12 @@ public class HiveSubQRemoveRelBuilder {
       default:
         postCondition = condition;
       }
-      if(createSemiJoin) {
+      if(semiJoinType != null) {
         join = correlateFactory.createCorrelate(left.rel, right.rel, id,
-            requiredColumns, JoinRelType.SEMI);
+                requiredColumns, semiJoinType);
       } else {
         join = correlateFactory.createCorrelate(left.rel, right.rel, id,
-            requiredColumns, joinType);
+                requiredColumns, joinType);
       }
     } else {
       join = joinFactory.createJoin(left.rel, right.rel, condition,
@@ -1161,7 +1162,7 @@ public class HiveSubQRemoveRelBuilder {
    * variables. */
   public HiveSubQRemoveRelBuilder join(JoinRelType joinType, RexNode condition,
                                        Set<CorrelationId> variablesSet) {
-    return join(joinType, condition, variablesSet, false);
+    return join(joinType, condition, variablesSet, null);
   }
 
   /** Creates a {@link org.apache.calcite.rel.core.Join} using USING syntax.
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAntiJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAntiJoin.java
new file mode 100644
index 0000000..176cb70
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAntiJoin.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+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.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRulesRegistry;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class HiveAntiJoin extends Join implements HiveRelNode {
+
+  // The joinFilter holds the residual filter which is used during post processing.
+  // These are the join conditions that are not part of the join key.
+  private final RexNode joinFilter;
+
+  public static HiveAntiJoin getAntiJoin(
+          RelOptCluster cluster,
+          RelTraitSet traitSet,
+          RelNode left,
+          RelNode right,
+          RexNode condition) {
+    try {
+      return new HiveAntiJoin(cluster, traitSet, left, right, condition);
+    } catch (CalciteSemanticException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  protected HiveAntiJoin(RelOptCluster cluster,
+                         RelTraitSet traitSet,
+                         RelNode left,
+                         RelNode right,
+                         RexNode condition) throws CalciteSemanticException {
+    super(cluster, traitSet, left, right, condition, JoinRelType.ANTI, Sets.newHashSet());
+    final List<RelDataTypeField> systemFieldList = ImmutableList.of();
+    List<List<RexNode>> joinKeyExprs = new ArrayList<List<RexNode>>();
+    List<Integer> filterNulls = new ArrayList<Integer>();
+    for (int i=0; i<this.getInputs().size(); i++) {
+      joinKeyExprs.add(new ArrayList<>());
+    }
+    this.joinFilter = HiveRelOptUtil.splitHiveJoinCondition(systemFieldList, this.getInputs(),
+            this.getCondition(), joinKeyExprs, filterNulls, null);
+  }
+
+  public RexNode getJoinFilter() {
+    return joinFilter;
+  }
+
+  @Override
+  public HiveAntiJoin copy(RelTraitSet traitSet, RexNode condition,
+                           RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+    try {
+      HiveAntiJoin antiJoin = new HiveAntiJoin(getCluster(), traitSet, left, right, condition);
+      // If available, copy state to registry for optimization rules
+      HiveRulesRegistry registry = antiJoin.getCluster().getPlanner().getContext().unwrap(HiveRulesRegistry.class);
+      if (registry != null) {
+        registry.copyPushedPredicates(this, antiJoin);
+      }
+      return antiJoin;
+    } catch (CalciteSemanticException e) {
+      // Semantic error not possible. Must be a bug. Convert to
+      // internal error.
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override
+  public void implement(Implementor implementor) {
+  }
+
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAntiSemiJoinRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAntiSemiJoinRule.java
new file mode 100644
index 0000000..14a64c3
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAntiSemiJoinRule.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Planner rule that converts a join plus filter to anti join.
+ */
+public class HiveAntiSemiJoinRule extends RelOptRule {
+  protected static final Logger LOG = LoggerFactory.getLogger(HiveAntiSemiJoinRule.class);
+  public static final HiveAntiSemiJoinRule INSTANCE = new HiveAntiSemiJoinRule();
+
+  //    HiveProject(fld=[$0])
+  //      HiveFilter(condition=[IS NULL($1)])
+  //        HiveJoin(condition=[=($0, $1)], joinType=[left], algorithm=[none], cost=[not available])
+  //
+  // TO
+  //
+  //    HiveProject(fld_tbl=[$0])
+  //      HiveAntiJoin(condition=[=($0, $1)], joinType=[anti])
+  //
+  public HiveAntiSemiJoinRule() {
+    super(operand(Project.class, operand(Filter.class, operand(Join.class, RelOptRule.any()))),
+            "HiveJoinWithFilterToAntiJoinRule:filter");
+  }
+
+  // is null filter over a left join.
+  public void onMatch(final RelOptRuleCall call) {
+    final Project project = call.rel(0);
+    final Filter filter = call.rel(1);
+    final Join join = call.rel(2);
+    perform(call, project, filter, join);
+  }
+
+  protected void perform(RelOptRuleCall call, Project project, Filter filter, Join join) {
+    LOG.debug("Start Matching HiveAntiJoinRule");
+
+    //TODO : Need to support this scenario.
+    //https://issues.apache.org/jira/browse/HIVE-23991
+    if (join.getCondition().isAlwaysTrue()) {
+      return;
+    }
+
+    //We support conversion from left outer join only.
+    if (join.getJoinType() != JoinRelType.LEFT) {
+      return;
+    }
+
+    assert (filter != null);
+
+    List<RexNode> filterList = getResidualFilterNodes(filter, join);
+    if (filterList == null) {
+      return;
+    }
+
+    // If any projection is there from right side, then we can not convert to anti join.
+    boolean hasProjection = HiveCalciteUtil.hasAnyExpressionFromRightSide(join, project.getProjects());
+    if (hasProjection) {
+      return;
+    }
+
+    LOG.debug("Matched HiveAntiJoinRule");
+
+    // Build anti join with same left, right child and condition as original left outer join.
+    Join anti = HiveAntiJoin.getAntiJoin(join.getLeft().getCluster(), join.getLeft().getTraitSet(),
+            join.getLeft(), join.getRight(), join.getCondition());
+    RelNode newProject;
+    if (filterList.isEmpty()) {
+      newProject = project.copy(project.getTraitSet(), anti, project.getProjects(), project.getRowType());
+    } else {
+      // Collate the filter condition using AND as the filter was decomposed based
+      // on AND condition (RelOptUtil.conjunctions).
+      RexNode condition = filterList.size() == 1 ? filterList.get(0) :
+              join.getCluster().getRexBuilder().makeCall(SqlStdOperatorTable.AND, filterList);
+      Filter newFilter = filter.copy(filter.getTraitSet(), anti, condition);
+      newProject = project.copy(project.getTraitSet(), newFilter, project.getProjects(), project.getRowType());
+    }
+    call.transformTo(newProject);
+  }
+
+  /**
+   * Extracts the non-null filter conditions from given filter node.
+   *
+   * @param filter The filter condition to be checked.
+   * @param join Join node whose right side has to be searched.
+   * @return null : Anti join condition is not matched for filter.
+   *         Empty list : No residual filter conditions present.
+   *         Valid list containing the filter to be applied after join.
+   */
+  private List<RexNode> getResidualFilterNodes(Filter filter, Join join) {
+    // 1. If null filter is not present from right side then we can not convert to anti join.
+    // 2. If any non-null filter is present from right side, we can not convert it to anti join.
+    // 3. Keep other filters which needs to be executed after join.
+    // 4. The filter conditions are decomposed on AND conditions only.
+    //TODO If some conditions like (fld1 is null or fld2 is null) present, it will not be considered for conversion.
+    //https://issues.apache.org/jira/browse/HIVE-23992
+    List<RexNode> aboveFilters = RelOptUtil.conjunctions(filter.getCondition());
+    boolean hasNullFilterOnRightSide = false;
+    List<RexNode> filterList = new ArrayList<>();
+    for (RexNode filterNode : aboveFilters) {
+      if (filterNode.getKind() == SqlKind.IS_NULL) {
+        // Null filter from right side table can be removed and its a pre-condition for anti join conversion.
+        if (HiveCalciteUtil.hasAnyExpressionFromRightSide(join, Collections.singletonList(filterNode))) {
+          hasNullFilterOnRightSide = true;
+        } else {
+          filterList.add(filterNode);
+        }
+      } else {
+        if (HiveCalciteUtil.hasAnyExpressionFromRightSide(join, Collections.singletonList(filterNode))) {
+          // If some non null condition is present from right side, we can not convert the join to anti join as
+          // anti join does not project the fields from right side.
+          return null;
+        } else {
+          filterList.add(filterNode);
+        }
+      }
+    }
+
+    if (!hasNullFilterOnRightSide) {
+      return null;
+    }
+    return filterList;
+  }
+}
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 b2ff255..3090294 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil.JoinLeafPredicateInfo;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil.JoinPredicateInfo;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 
@@ -56,6 +57,9 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
   public static final HiveJoinAddNotNullRule INSTANCE_SEMIJOIN =
       new HiveJoinAddNotNullRule(HiveSemiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
 
+  public static final HiveJoinAddNotNullRule INSTANCE_ANTIJOIN =
+      new HiveJoinAddNotNullRule(HiveAntiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
+
   private final FilterFactory filterFactory;
 
   //~ Constructors -----------------------------------------------------------
@@ -74,7 +78,15 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
   @Override
   public void onMatch(RelOptRuleCall call) {
     Join join = call.rel(0);
-    if (join.getJoinType() == JoinRelType.FULL || join.getCondition().isAlwaysTrue()) {
+
+    // For anti join case add the not null on right side even if the condition is always true.
+    // This is done because during execution, anti join expect the right side to be empty and
+    // if we dont put null check on right, for null only right side table and condition always
+    // true, execution will produce 0 records as the post processing to filter out null value
+    // is not done for always true conditions during execution.
+    // eg  select * from left_tbl where (select 1 from all_null_right limit 1) is null
+    if (join.getJoinType() == JoinRelType.FULL ||
+            (join.getJoinType() != JoinRelType.ANTI && join.getCondition().isAlwaysTrue())) {
       return;
     }
 
@@ -92,7 +104,7 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
     Set<String> rightPushedPredicates = Sets.newHashSet(registry.getPushedPredicates(join, 1));
 
     boolean genPredOnLeft = join.getJoinType() == JoinRelType.RIGHT || join.getJoinType() == JoinRelType.INNER || join.isSemiJoin();
-    boolean genPredOnRight = join.getJoinType() == JoinRelType.LEFT || join.getJoinType() == JoinRelType.INNER || join.isSemiJoin();
+    boolean genPredOnRight = join.getJoinType() == JoinRelType.LEFT || join.getJoinType() == JoinRelType.INNER || join.isSemiJoin()|| join.getJoinType() == JoinRelType.ANTI;
 
     RexNode newLeftPredicate = getNewPredicate(join, registry, joinPredInfo, leftPushedPredicates, genPredOnLeft, 0);
     RexNode newRightPredicate = getNewPredicate(join, registry, joinPredInfo, rightPushedPredicates, genPredOnRight, 1);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java
index 8acecfc..0c533e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java
@@ -85,6 +85,11 @@ public class HiveJoinConstraintsRule extends RelOptRule {
     final RelNode rightInput = join.getRight();
     final RexNode cond = join.getCondition();
 
+    //TODO:https://issues.apache.org/jira/browse/HIVE-23920
+    if (joinType == JoinRelType.ANTI) {
+      return;
+    }
+
     // 1) If it is an inner, check whether project only uses columns from one side.
     // That side will need to be the FK side.
     // If it is a left outer, left will be the FK side.
@@ -100,7 +105,8 @@ public class HiveJoinConstraintsRule extends RelOptRule {
     // These boolean values represent corresponding left, right input which is potential FK
     boolean leftInputPotentialFK = topRefs.intersects(leftBits);
     boolean rightInputPotentialFK = topRefs.intersects(rightBits);
-    if (leftInputPotentialFK && rightInputPotentialFK && (joinType == JoinRelType.INNER || joinType == JoinRelType.SEMI)) {
+    if (leftInputPotentialFK && rightInputPotentialFK &&
+            (joinType == JoinRelType.INNER || joinType == JoinRelType.SEMI)) {
       // Both inputs are referenced. Before making a decision, try to swap
       // references in join condition if it is an inner join, i.e. if a join
       // condition column is referenced above the join, then we can just
@@ -183,6 +189,7 @@ public class HiveJoinConstraintsRule extends RelOptRule {
     switch (joinType) {
     case SEMI:
     case INNER:
+    //case ANTI: //TODO:https://issues.apache.org/jira/browse/HIVE-23920
       if (leftInputPotentialFK && rightInputPotentialFK) {
         // Bails out as it references columns from both sides (or no columns)
         // and there is nothing to transform
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
index 17984ac..c9742aa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
@@ -24,6 +24,8 @@ import static org.apache.calcite.plan.RelOptRule.some;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 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.rules.JoinProjectTransposeRule;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
@@ -133,6 +135,12 @@ public final class HiveJoinProjectTransposeRule {
 
     public void onMatch(RelOptRuleCall call) {
       //TODO: this can be removed once CALCITE-3824 is released
+      Join joinRel = call.rel(0);
+
+      //TODO:https://issues.apache.org/jira/browse/HIVE-23921
+      if (joinRel.getJoinType() == JoinRelType.ANTI) {
+        return;
+      }
       HiveProject proj;
       if (hasLeftChild(call)) {
         proj = call.rel(1);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
index 3569265..3a2bf82 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
@@ -42,6 +42,7 @@ import org.apache.calcite.util.Util;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
@@ -70,6 +71,9 @@ public class HiveJoinPushTransitivePredicatesRule extends RelOptRule {
   public static final HiveJoinPushTransitivePredicatesRule INSTANCE_SEMIJOIN =
           new HiveJoinPushTransitivePredicatesRule(HiveSemiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
 
+  public static final HiveJoinPushTransitivePredicatesRule INSTANCE_ANTIJOIN =
+          new HiveJoinPushTransitivePredicatesRule(HiveAntiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
+
   private final FilterFactory filterFactory;
 
   public HiveJoinPushTransitivePredicatesRule(Class<? extends Join> clazz,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectJoinTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectJoinTransposeRule.java
index 545255c..3fd76f5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectJoinTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectJoinTransposeRule.java
@@ -23,6 +23,7 @@ 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.Project;
 import org.apache.calcite.rel.rules.PushProjector;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -76,7 +77,7 @@ public class HiveProjectJoinTransposeRule extends RelOptRule {
     Project origProj = call.rel(0);
     final Join join = call.rel(1);
 
-    if (join.isSemiJoin()) {
+    if (join.getJoinType() == JoinRelType.SEMI || join.getJoinType() == JoinRelType.ANTI) {
       return; // TODO: support SemiJoin
     }
     // locate all fields referenced in the projection and join condition;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java
index ab56ce8..8f61da7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelShuttleImpl;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIntersect;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
@@ -1239,12 +1240,16 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
       valueGenerator = false;
     }
 
-    if(oldInput instanceof LogicalCorrelate
-        && ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI
-        &&  !cm.mapRefRelToCorRef.containsKey(rel)) {
+    boolean isSemiJoin = false;
+    if (oldInput instanceof LogicalCorrelate) {
+      isSemiJoin = ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI ||
+              ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.ANTI;
+    }
+
+    if(isSemiJoin && !cm.mapRefRelToCorRef.containsKey(rel)) {
       // this conditions need to be pushed into semi-join since this condition
       // corresponds to IN
-      HiveSemiJoin join = ((HiveSemiJoin)frame.r);
+      Join join = ((Join)frame.r);
       final List<RexNode> conditions = new ArrayList<>();
       RexNode joinCond = join.getCondition();
       conditions.add(joinCond);
@@ -1252,8 +1257,14 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
       final RexNode condition =
           RexUtil.composeConjunction(rexBuilder, conditions, false);
 
-      RelNode newRel = HiveSemiJoin.getSemiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
-          join.getLeft(), join.getRight(), condition);
+      RelNode newRel;
+      if (((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI) {
+        newRel = HiveSemiJoin.getSemiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
+                join.getLeft(), join.getRight(), condition);
+      } else {
+        newRel = HiveAntiJoin.getAntiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
+                join.getLeft(), join.getRight(), condition);
+      }
 
       return register(rel, newRel, frame.oldToNewOutputs, frame.corDefOutputs);
     }
@@ -1311,9 +1322,13 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
       valueGenerator = false;
     }
 
-    if(oldInput instanceof LogicalCorrelate
-        && ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI
-        &&  !cm.mapRefRelToCorRef.containsKey(rel)) {
+    boolean isSemiJoin = false;
+    if (oldInput instanceof LogicalCorrelate) {
+      isSemiJoin = ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI ||
+              ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.ANTI;
+    }
+
+    if(isSemiJoin && !cm.mapRefRelToCorRef.containsKey(rel)) {
       // this conditions need to be pushed into semi-join since this condition
       // corresponds to IN
       HiveSemiJoin join = ((HiveSemiJoin)frame.r);
@@ -1323,8 +1338,15 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
       conditions.add(decorrelateExpr(rel.getCondition(), valueGenerator));
       final RexNode condition =
           RexUtil.composeConjunction(rexBuilder, conditions, false);
-      RelNode newRel = HiveSemiJoin.getSemiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
-          join.getLeft(), join.getRight(), condition);
+
+      RelNode newRel;
+      if (((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI) {
+        newRel = HiveSemiJoin.getSemiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
+                join.getLeft(), join.getRight(), condition);
+      } else {
+        newRel = HiveAntiJoin.getAntiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
+                join.getLeft(), join.getRight(), condition);
+      }
       return register(rel, newRel, frame.oldToNewOutputs, frame.corDefOutputs);
     }
 
@@ -1447,14 +1469,18 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
     RelNode newJoin = null;
 
     // this indicates original query was either correlated EXISTS or IN
-    if(rel.getJoinType() == JoinRelType.SEMI) {
+    if(rel.getJoinType() == JoinRelType.SEMI || rel.getJoinType() == JoinRelType.ANTI) {
       final List<Integer> leftKeys = new ArrayList<Integer>();
       final List<Integer> rightKeys = new ArrayList<Integer>();
 
       RelNode[] inputRels = new RelNode[] {leftFrame.r, rightFrame.r};
-      newJoin = HiveSemiJoin.getSemiJoin(rel.getCluster(),
-          rel.getCluster().traitSetOf(HiveRelNode.CONVENTION), leftFrame.r, rightFrame.r, condition);
-
+      if (rel.getJoinType() == JoinRelType.ANTI) {
+        newJoin = HiveAntiJoin.getAntiJoin(rel.getCluster(),
+                rel.getCluster().traitSetOf(HiveRelNode.CONVENTION), leftFrame.r, rightFrame.r, condition);
+      } else {
+        newJoin = HiveSemiJoin.getSemiJoin(rel.getCluster(),
+                rel.getCluster().traitSetOf(HiveRelNode.CONVENTION), leftFrame.r, rightFrame.r, condition);
+      }
     } else {
       // Right input positions are shifted by newLeftFieldCount.
       for (int i = 0; i < oldRightFieldCount; i++) {
@@ -1610,7 +1636,7 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
     if(oldInput == null) {
       if(currentRel.getInputs().size() == 1 && currentRel.getInput(0) instanceof LogicalCorrelate) {
         final Frame newFrame = map.get(currentRel.getInput(0));
-        if(newFrame.r instanceof HiveSemiJoin) {
+        if(newFrame.r instanceof HiveSemiJoin || newFrame.r instanceof HiveAntiJoin) {
           int oldFieldSize = currentRel.getInput(0).getRowType().getFieldCount();
           int newOrd = newFrame.r.getRowType().getFieldCount() + oldOrdinalNo - oldFieldSize;
           return new RexInputRef(newOrd, oldInputRef.getType());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveGBYSemiJoinRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveGBYSemiJoinRule.java
index 82704a2..abad7db 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveGBYSemiJoinRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveGBYSemiJoinRule.java
@@ -20,17 +20,17 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Planner rule that removes a {@code Aggregate} from a HiveSemiJoin
+ * Planner rule that removes a {@code Aggregate} from a HiveSemiJoin/HiveAntiJoin
  * right input.
  */
 public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
@@ -41,7 +41,7 @@ public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
 
   public HiveRemoveGBYSemiJoinRule() {
     super(
-        operand(HiveSemiJoin.class,
+        operand(Join.class,
             some(
                 operand(RelNode.class, any()),
                 operand(Aggregate.class, any()))),
@@ -49,9 +49,11 @@ public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
   }
 
   @Override public void onMatch(RelOptRuleCall call) {
-    final HiveSemiJoin semijoin= call.rel(0);
+    final Join join = call.rel(0);
 
-    assert semijoin.getJoinType() == JoinRelType.SEMI;
+    if (join.getJoinType() != JoinRelType.SEMI && join.getJoinType() != JoinRelType.ANTI) {
+      return;
+    }
 
     final RelNode left = call.rel(1);
     final Aggregate rightAggregate= call.rel(2);
@@ -69,7 +71,7 @@ public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
     if(!rightAggregate.getAggCallList().isEmpty()) {
       return;
     }
-    final JoinInfo joinInfo = semijoin.analyzeCondition();
+    final JoinInfo joinInfo = join.analyzeCondition();
 
     boolean shouldTransform = joinInfo.rightSet().equals(
         ImmutableBitSet.range(rightAggregate.getGroupCount()));
@@ -77,9 +79,15 @@ public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
       final RelBuilder relBuilder = call.builder();
       RelNode newRightInput = relBuilder.project(relBuilder.push(rightAggregate.getInput()).
           fields(rightAggregate.getGroupSet().asList())).build();
-      RelNode newSemiJoin = call.builder().push(left).push(newRightInput)
-          .semiJoin(semijoin.getCondition()).build();
-      call.transformTo(newSemiJoin);
+      RelNode newJoin;
+      if (join.getJoinType() == JoinRelType.SEMI) {
+        newJoin = call.builder().push(left).push(newRightInput)
+                .semiJoin(join.getCondition()).build();
+      } else {
+        newJoin = call.builder().push(left).push(newRightInput)
+                .antiJoin(join.getCondition()).build();
+      }
+      call.transformTo(newJoin);
     }
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSubQueryRemoveRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSubQueryRemoveRule.java
index d2d6f70..a508b50 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSubQueryRemoveRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSubQueryRemoveRule.java
@@ -63,6 +63,8 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_CONVERT_ANTI_JOIN;
+
 /**
  * NOTE: this rule is replicated from Calcite's SubqueryRemoveRule
  * Transform that converts IN, EXISTS and scalar sub-queries into joins.
@@ -405,7 +407,7 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
         builder.push(e.rel);
       }
     }
-
+    boolean isCandidateForAntiJoin = false;
     // First, the cross join
     switch (logic) {
     case TRUE_FALSE_UNKNOWN:
@@ -414,6 +416,13 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
       // null keys we do not need to generate count(*), count(c)
       if (e.getKind() == SqlKind.EXISTS) {
         logic = RelOptUtil.Logic.TRUE_FALSE;
+        if (conf.getBoolVar(HiveConf.ConfVars.HIVE_CONVERT_ANTI_JOIN)) {
+          //TODO : As of now anti join is first converted to left outer join
+          // and then converted to anti join.
+          //logic = RelOptUtil.Logic.FALSE;
+
+          isCandidateForAntiJoin = true;
+        }
         break;
       }
       builder.aggregate(builder.groupKey(), builder.count(false, "c"),
@@ -435,8 +444,13 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
     String trueLiteral = "literalTrue";
     switch (logic) {
     case TRUE:
+    case FALSE:
       if (fields.isEmpty()) {
-        builder.project(builder.alias(builder.literal(true), trueLiteral));
+        if (logic == RelOptUtil.Logic.TRUE) {
+          builder.project(builder.alias(builder.literal(true), trueLiteral));
+        } else {
+          builder.project(builder.alias(builder.literal(false), "literalFalse"));
+        }
         if (!variablesSet.isEmpty() && (e.getKind() == SqlKind.EXISTS
             || e.getKind() == SqlKind.IN)) {
           // avoid adding group by for correlated IN/EXISTS queries
@@ -459,7 +473,12 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
     default:
       fields.add(builder.alias(builder.literal(true), trueLiteral));
       builder.project(fields);
-      builder.distinct();
+      // If, not-exists is first converted to left outer join with null
+      // filter and then to anti join, then the distinct clause is added
+      // later during semi/anti join processing at genMapGroupByForSemijoin.
+      if (!isCandidateForAntiJoin || variablesSet.isEmpty()) {
+        builder.distinct();
+      }
     }
     builder.as("dt");
     final List<RexNode> conditions = new ArrayList<>();
@@ -468,8 +487,11 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
     }
     switch (logic) {
     case TRUE:
-      builder.join(JoinRelType.INNER, builder.and(conditions), variablesSet, true);
+      builder.join(JoinRelType.INNER, builder.and(conditions), variablesSet, JoinRelType.SEMI);
       return builder.literal(true);
+    case FALSE:
+      builder.join(JoinRelType.LEFT, builder.and(conditions), variablesSet, JoinRelType.ANTI);
+      return builder.literal(false);
     }
     builder.join(JoinRelType.LEFT, builder.and(conditions), variablesSet);
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/EstimateUniqueKeys.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/EstimateUniqueKeys.java
index e36259c..0da4eaa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/EstimateUniqueKeys.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/EstimateUniqueKeys.java
@@ -35,6 +35,7 @@ import org.apache.calcite.util.ImmutableBitSet;
 import com.google.common.collect.ImmutableSet;
 import org.apache.calcite.util.Util;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
@@ -269,6 +270,12 @@ public final class EstimateUniqueKeys {
     return getUniqueKeys(rel.getLeft());
   }
 
+  private static Set<ImmutableBitSet> getUniqueKeys(HiveAntiJoin rel) {
+    // only return the unique keys from the LHS since a anti join only
+    // returns the LHS
+    return getUniqueKeys(rel.getLeft());
+  }
+
   private static Set<ImmutableBitSet> getUniqueKeys(HiveAggregate rel) {
     // group by keys form a unique key
     return ImmutableSet.of(rel.getGroupSet());
@@ -297,6 +304,8 @@ public final class EstimateUniqueKeys {
       return getUniqueKeys((HiveJoin) rel);
     } else if (rel instanceof HiveSemiJoin) {
       return getUniqueKeys((HiveSemiJoin) rel);
+    } else if (rel instanceof HiveAntiJoin) {
+      return getUniqueKeys((HiveAntiJoin) rel);
     } else if (rel instanceof HiveAggregate) {
       return getUniqueKeys((HiveAggregate) rel);
     } else if (rel instanceof SetOp) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistinctRowCount.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistinctRowCount.java
index af96354..70afebb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistinctRowCount.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistinctRowCount.java
@@ -39,6 +39,7 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.NumberUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveCost;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
@@ -79,6 +80,13 @@ public class HiveRelMdDistinctRowCount extends RelMdDistinctRowCount {
     return super.getDistinctRowCount(rel, mq, groupKey, predicate);
   }
 
+  public Double getDistinctRowCount(HiveAntiJoin rel, RelMetadataQuery mq, ImmutableBitSet groupKey,
+                                    RexNode predicate) {
+    //TODO : Currently calcite does not support this.
+    // https://issues.apache.org/jira/browse/HIVE-23933
+    return super.getDistinctRowCount(rel, mq, groupKey, predicate);
+  }
+
   public Double getDistinctRowCount(HiveJoin rel, RelMetadataQuery mq, ImmutableBitSet groupKey,
       RexNode predicate) {
     return getJoinDistinctRowCount(mq, rel, rel.getJoinType(),
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
index 0527e2b..66c6888 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
@@ -416,6 +416,7 @@ public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Pred
       case INNER:
       case LEFT:
       case SEMI:
+      case ANTI:
         infer(leftPreds, allExprsDigests, inferredPredicates,
             nonFieldsPredicates, includeEqualityInference,
             joinType == JoinRelType.LEFT ? rightFieldsBitSet
@@ -454,7 +455,8 @@ public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Pred
         }
       }
 
-      if ((joinType == JoinRelType.INNER || joinType == JoinRelType.SEMI) && !nonFieldsPredicates.isEmpty()) {
+      if ((joinType == JoinRelType.INNER || joinType == JoinRelType.SEMI) &&
+              !nonFieldsPredicates.isEmpty()) {
         // Predicates without field references can be pushed to both inputs
         final Set<String> leftPredsSet = new HashSet<String>(
                 Lists.transform(leftPreds, HiveCalciteUtil.REX_STR_FN));
@@ -480,6 +482,7 @@ public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Pred
         return RelOptPredicateList.of(rexBuilder, Iterables.concat(leftPreds, leftInferredPredicates),
           leftInferredPredicates, rightInferredPredicates);
       case LEFT:
+      case ANTI:
         return RelOptPredicateList.of(rexBuilder,
           leftPreds, EMPTY_LIST, rightInferredPredicates);
       case RIGHT:
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
index 82e4cc1..a5829ec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil.PKFKJoinInfo;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import org.apache.hadoop.hive.ql.plan.ColStatistics;
@@ -118,6 +119,15 @@ public class HiveRelMdRowCount extends RelMdRowCount {
   }
 
   public Double getRowCount(HiveSemiJoin rel, RelMetadataQuery mq) {
+    return getRowCountInt(rel, mq);
+  }
+
+  public Double getRowCount(HiveAntiJoin rel, RelMetadataQuery mq) {
+    return getRowCountInt(rel, mq);
+  }
+
+  private Double getRowCountInt(Join rel, RelMetadataQuery mq) {
+    assert rel.getJoinType() == JoinRelType.SEMI || rel.getJoinType() == JoinRelType.ANTI;
     PKFKRelationInfo pkfk = analyzeJoinForPKFK(rel, mq);
     if (pkfk != null) {
       double selectivity = pkfk.pkInfo.selectivity * pkfk.ndvScalingFactor;
@@ -125,9 +135,20 @@ public class HiveRelMdRowCount extends RelMdRowCount {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Identified Primary - Foreign Key relation: {} {}", RelOptUtil.toString(rel), pkfk);
       }
-      return pkfk.fkInfo.rowCount * selectivity;
+      if (rel.getJoinType() == JoinRelType.ANTI) {
+        return pkfk.fkInfo.rowCount * (1 - selectivity);
+      } else {
+        return pkfk.fkInfo.rowCount * selectivity;
+      }
+    }
+
+    //TODO : Need to handle anti join in calcite
+    // https://issues.apache.org/jira/browse/HIVE-23933
+    if (rel.getJoinType() == JoinRelType.ANTI) {
+      return mq.getRowCount(rel.getLeft()) - super.getRowCount(rel, mq);
+    } else {
+      return super.getRowCount(rel, mq);
     }
-    return super.getRowCount(rel, mq);
   }
 
   @Override
@@ -248,7 +269,7 @@ public class HiveRelMdRowCount extends RelMdRowCount {
     // @todo: remove this. 8/28/14 hb
     // for now adding because RelOptUtil.classifyFilters has an assertion about
     // column counts that is not true for semiJoins.
-    if (joinRel.isSemiJoin()) {
+    if (joinRel.isSemiJoin() || joinRel.getJoinType() == JoinRelType.ANTI) {
       return null;
     }
 
@@ -355,7 +376,7 @@ public class HiveRelMdRowCount extends RelMdRowCount {
    */
   public static Pair<PKFKRelationInfo, RexNode> constraintsBasedAnalyzeJoinForPKFK(Join join, RelMetadataQuery mq) {
 
-    if (join.isSemiJoin()) {
+    if (join.isSemiJoin() || join.getJoinType() == JoinRelType.ANTI) {
       // TODO: Support semijoin
       return null;
     }
@@ -390,9 +411,9 @@ public class HiveRelMdRowCount extends RelMdRowCount {
       return null;
     }
 
-    boolean leftIsKey = (join.getJoinType() == JoinRelType.INNER || join.isSemiJoin() || join.getJoinType() == JoinRelType.RIGHT)
+    boolean leftIsKey = (join.getJoinType() == JoinRelType.INNER || join.isSemiJoin() || join.getJoinType() == JoinRelType.ANTI || join.getJoinType() == JoinRelType.RIGHT)
         && leftInputResult.isPkFkJoin;
-    boolean rightIsKey = (join.getJoinType() == JoinRelType.INNER || join.isSemiJoin() || join.getJoinType() == JoinRelType.LEFT)
+    boolean rightIsKey = (join.getJoinType() == JoinRelType.INNER || join.isSemiJoin() || join.getJoinType() == JoinRelType.ANTI || join.getJoinType() == JoinRelType.LEFT)
         && rightInputResult.isPkFkJoin;
     if (!leftIsKey && !rightIsKey) {
       // Nothing to do here, bail out
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
index 1724ab1..2c36d8f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
@@ -64,8 +64,12 @@ public class HiveRelMdSelectivity extends RelMdSelectivity {
   }
 
   public Double getSelectivity(Join j, RelMetadataQuery mq, RexNode predicate) {
-    if (j.getJoinType().equals(JoinRelType.INNER) || j.isSemiJoin()) {
-      return computeInnerJoinSelectivity(j, mq, predicate);
+    if (j.getJoinType().equals(JoinRelType.INNER) || j.isSemiJoin() || j.getJoinType().equals(JoinRelType.ANTI)) {
+      Double selectivity =  computeInnerJoinSelectivity(j, mq, predicate);
+      if (j.getJoinType().equals(JoinRelType.ANTI)) {
+        return 1 - selectivity;
+      }
+      return selectivity;
     } else if (j.getJoinType().equals(JoinRelType.LEFT) ||
             j.getJoinType().equals(JoinRelType.RIGHT)) {
       double left = mq.getRowCount(j.getLeft());
@@ -142,7 +146,7 @@ public class HiveRelMdSelectivity extends RelMdSelectivity {
         ndvEstimate = exponentialBackoff(peLst, colStatMap);
       }
 
-      if (j.isSemiJoin()) {
+      if (j.isSemiJoin() || (j.getJoinType().equals(JoinRelType.ANTI))) {
         ndvEstimate = Math.min(mq.getRowCount(j.getLeft()),
             ndvEstimate);
       } else if (j instanceof HiveJoin) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
index d5a4e36..0d99f20 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.stats;
 import java.util.List;
 
 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.metadata.ReflectiveRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMdSize;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -29,6 +31,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableNullableList;
 import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
@@ -88,6 +91,15 @@ public class HiveRelMdSize extends RelMdSize {
   }
 
   public List<Double> averageColumnSizes(HiveSemiJoin rel, RelMetadataQuery mq) {
+    return averageColumnSizesInt(rel, mq);
+  }
+
+  public List<Double> averageColumnSizes(HiveAntiJoin rel, RelMetadataQuery mq) {
+    return averageColumnSizesInt(rel, mq);
+  }
+
+  private List<Double> averageColumnSizesInt(Join rel, RelMetadataQuery mq) {
+    assert rel.getJoinType() == JoinRelType.SEMI || rel.getJoinType() == JoinRelType.ANTI;
     final RelNode left = rel.getLeft();
     final List<Double> lefts =
         mq.getAverageColumnSizes(left);
@@ -96,9 +108,7 @@ public class HiveRelMdSize extends RelMdSize {
     }
     final int fieldCount = rel.getRowType().getFieldCount();
     Double[] sizes = new Double[fieldCount];
-    if (lefts != null) {
-      lefts.toArray(sizes);
-    }
+    lefts.toArray(sizes);
     return ImmutableNullableList.copyOf(sizes);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
index cbeec35..8abc28a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
@@ -175,6 +175,9 @@ public class ASTBuilder {
     case FULL:
       b = ASTBuilder.construct(HiveParser.TOK_FULLOUTERJOIN, "TOK_FULLOUTERJOIN");
       break;
+    case ANTI:
+      b = ASTBuilder.construct(HiveParser.TOK_LEFTANTISEMIJOIN, "TOK_LEFTANTISEMIJOIN");
+      break;
     }
 
     b.add(left).add(right).add(cond);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index cee9f8b..072a27d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -409,7 +409,7 @@ public class ASTConverter {
       QueryBlockInfo right = convertSource(join.getRight());
       s = new Schema(left.schema, right.schema);
       ASTNode cond = join.getCondition().accept(new RexVisitor(s, false, r.getCluster().getRexBuilder()));
-      boolean semiJoin = join.isSemiJoin();
+      boolean semiJoin = join.isSemiJoin() || join.getJoinType() == JoinRelType.ANTI;
       if (join.getRight() instanceof Join && !semiJoin) {
           // should not be done for semijoin since it will change the semantics
         // Invert join inputs; this is done because otherwise the SemanticAnalyzer
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index fe4ecf0..d414966 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
@@ -287,12 +288,13 @@ public class PlanModifierForASTConv {
       // But we only need the additional project if the left child
       // is another join too; if it is not, ASTConverter will swap
       // the join inputs, leaving the join operator on the left.
-      // we also do it if parent is HiveSemiJoin since ASTConverter won't
-      // swap inputs then
+      // we also do it if parent is HiveSemiJoin or HiveAntiJoin since
+      // ASTConverter won't swap inputs then.
       // This will help triggering multijoin recognition methods that
       // are embedded in SemanticAnalyzer.
       if (((Join) parent).getRight() == joinNode &&
-            (((Join) parent).getLeft() instanceof Join || parent instanceof HiveSemiJoin) ) {
+            (((Join) parent).getLeft() instanceof Join || parent instanceof HiveSemiJoin
+                  || parent instanceof HiveAntiJoin) ) {
         validParent = false;
       }
     } else if (parent instanceof SetOp) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveOpConverter.java
index bc2b742..905a4f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveOpConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveOpConverter.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.LimitOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
@@ -98,7 +99,9 @@ public class HiveOpConverter {
     } else if (rn instanceof HiveJoin) {
       return new JoinVisitor(this).visit((HiveJoin) rn);
     } else if (rn instanceof HiveSemiJoin) {
-      return new JoinVisitor(this).visit((HiveSemiJoin) rn);
+      return new JoinVisitor(this).visit(rn);
+    } else if (rn instanceof HiveAntiJoin) {
+      return new JoinVisitor(this).visit(rn);
     } else if (rn instanceof HiveFilter) {
       return new HiveFilterVisitor(this).visit((HiveFilter) rn);
     } else if (rn instanceof HiveSortLimit) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
index 8d9d5ae..3ad658c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
@@ -87,7 +88,8 @@ class JoinVisitor extends HiveRelNodeVisitor<RelNode> {
     // 3. Virtual columns
     Set<Integer> newVcolsInCalcite = new HashSet<Integer>();
     newVcolsInCalcite.addAll(inputs[0].vcolsInCalcite);
-    if (joinRel instanceof HiveMultiJoin || !((joinRel instanceof Join) && ((Join) joinRel).isSemiJoin())) {
+    if (joinRel instanceof HiveMultiJoin || !((joinRel instanceof Join) &&
+            ((((Join) joinRel).isSemiJoin()) || (((Join) joinRel).getJoinType() == JoinRelType.ANTI)))) {
       int shift = inputs[0].inputs.get(0).getSchema().getSignature().size();
       for (int i = 1; i < inputs.length; i++) {
         newVcolsInCalcite.addAll(HiveCalciteUtil.shiftVColsSet(inputs[i].vcolsInCalcite, shift));
@@ -116,6 +118,8 @@ class JoinVisitor extends HiveRelNodeVisitor<RelNode> {
       joinFilters = ((HiveMultiJoin)joinRel).getJoinFilters();
     } else if (joinRel instanceof HiveSemiJoin){
       joinFilters = ImmutableList.of(((HiveSemiJoin)joinRel).getJoinFilter());
+    } else if (joinRel instanceof HiveAntiJoin){
+      joinFilters = ImmutableList.of(((HiveAntiJoin)joinRel).getJoinFilter());
     } else {
       throw new SemanticException("Can't handle join type: " + joinRel.getClass().getName());
     }
@@ -159,12 +163,24 @@ class JoinVisitor extends HiveRelNodeVisitor<RelNode> {
       noOuterJoin = !hmj.isOuterJoin();
     } else {
       joinCondns = new JoinCondDesc[1];
-      semiJoin = (join instanceof Join) && ((Join) join).isSemiJoin();
+      JoinRelType joinRelType = JoinRelType.INNER;
+      if (join instanceof Join) {
+        joinRelType = ((Join) join).getJoinType();
+      }
       JoinType joinType;
-      if (semiJoin) {
-        joinType = JoinType.LEFTSEMI;
-      } else {
-        joinType = transformJoinType(((Join)join).getJoinType());
+      switch (joinRelType) {
+        case SEMI:
+          joinType = JoinType.LEFTSEMI;
+          semiJoin = true;
+          break;
+        case ANTI:
+          joinType = JoinType.ANTI;
+          semiJoin = true;
+          break;
+        default:
+          assert join instanceof Join;
+          joinType = transformJoinType(((Join)join).getJoinType());
+          semiJoin = false;
       }
       joinCondns[0] = new JoinCondDesc(new JoinCond(0, 1, joinType));
       noOuterJoin = joinType != JoinType.FULLOUTER && joinType != JoinType.LEFTOUTER
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
index c33f39d..2c35b9d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
@@ -266,7 +266,8 @@ public class CorrelationOptimizer extends Transform {
         if (pos == joinCond.getLeft()) {
           if (type == JoinDesc.INNER_JOIN ||
               type == JoinDesc.LEFT_OUTER_JOIN ||
-              type == JoinDesc.LEFT_SEMI_JOIN) {
+              type == JoinDesc.LEFT_SEMI_JOIN ||
+              type == JoinDesc.ANTI_JOIN) {
             Operator<? extends OperatorDesc> newCurrentRsOps = rsOps.get(joinCond.getRight());
             analyzeReduceSinkOperatorsOfJoinOperator(joinConds, rsOps, newCurrentRsOps,
                 correlatedRsOps);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index a690cd7..99b272c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -45,6 +45,9 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ConvertDecimal64ToDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorCoalesce;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DecimalColDivideDecimalScalar;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinAntiJoinLongOperator;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinAntiJoinMultiKeyOperator;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinAntiJoinStringOperator;
 import org.apache.hadoop.hive.ql.exec.vector.reducesink.*;
 import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFArgDesc;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
@@ -3416,6 +3419,10 @@ public class Vectorizer implements PhysicalPlanResolver {
       vectorMapJoinVariation = VectorMapJoinVariation.LEFT_SEMI;
       hashTableKind = HashTableKind.HASH_SET;
       break;
+    case JoinDesc.ANTI_JOIN:
+        vectorMapJoinVariation = VectorMapJoinVariation.LEFT_ANTI;
+        hashTableKind = HashTableKind.HASH_SET;
+        break;
     default:
       throw new HiveException("Unknown join type " + joinType);
     }
@@ -3438,6 +3445,9 @@ public class Vectorizer implements PhysicalPlanResolver {
       case LEFT_SEMI:
         opClass = VectorMapJoinLeftSemiLongOperator.class;
         break;
+      case LEFT_ANTI:
+        opClass = VectorMapJoinAntiJoinLongOperator.class;
+        break;
       case OUTER:
         opClass = VectorMapJoinOuterLongOperator.class;
         break;
@@ -3459,6 +3469,9 @@ public class Vectorizer implements PhysicalPlanResolver {
       case LEFT_SEMI:
         opClass = VectorMapJoinLeftSemiStringOperator.class;
         break;
+      case LEFT_ANTI:
+        opClass = VectorMapJoinAntiJoinStringOperator.class;
+        break;
       case OUTER:
         opClass = VectorMapJoinOuterStringOperator.class;
         break;
@@ -3480,6 +3493,9 @@ public class Vectorizer implements PhysicalPlanResolver {
       case LEFT_SEMI:
         opClass = VectorMapJoinLeftSemiMultiKeyOperator.class;
         break;
+      case LEFT_ANTI:
+        opClass = VectorMapJoinAntiJoinMultiKeyOperator.class;
+        break;
       case OUTER:
         opClass = VectorMapJoinOuterMultiKeyOperator.class;
         break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
index 4318c4b..1d1f967 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
@@ -2514,6 +2514,7 @@ public class StatsRulesProcFactory {
       case JoinDesc.INNER_JOIN:
       case JoinDesc.UNIQUE_JOIN:
       case JoinDesc.LEFT_SEMI_JOIN:
+      case JoinDesc.ANTI_JOIN:
         break;
       }
       colStats.setNumNulls(newNumNulls);
@@ -2606,6 +2607,17 @@ public class StatsRulesProcFactory {
           // max # of rows = rows from left side
           result = Math.min(rowCountParents.get(joinCond.getLeft()), result);
           break;
+        case JoinDesc.ANTI_JOIN:
+          long leftRowCount = rowCountParents.get(joinCond.getLeft());
+          if (leftRowCount < result) {
+            // Ideally the inner join count should be less than the left row count. but if its not calculated
+            // properly then we can assume whole of left table will be selected.
+            result = leftRowCount;
+          } else {
+            // The number of result should be left reduced by the number of rows matching (ie inner join count).
+            result = leftRowCount - result;
+          }
+          break;
         default:
           LOG.debug("Unhandled join type in stats estimation: " + joinCond.getType());
           break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 180b37e..679b6bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -181,6 +181,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException.Unsu
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveAlgorithmsConf;
 import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveVolcanoPlanner;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExcept;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
@@ -227,6 +228,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinConstraintsRule
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinProjectTransposeRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinPushTransitivePredicatesRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinToMultiJoinRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveAntiSemiJoinRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePartitionPruneRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePointLookupOptimizerRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePreFilteringRule;
@@ -393,6 +395,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
           HiveTableScan.class,
           HiveAggregate.class,
+          HiveAntiJoin.class,
           HiveExcept.class,
           HiveFilter.class,
           HiveIntersect.class,
@@ -1946,7 +1949,6 @@ public class CalcitePlanner extends SemanticAnalyzer {
         LOG.debug("Plan After Join Reordering:\n"
             + RelOptUtil.toString(calciteOptimizedPlan, SqlExplainLevel.ALL_ATTRIBUTES));
       }
-
       return calciteOptimizedPlan;
     }
 
@@ -2065,8 +2067,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
       }
       rules.add(HiveJoinAddNotNullRule.INSTANCE_JOIN);
       rules.add(HiveJoinAddNotNullRule.INSTANCE_SEMIJOIN);
+      rules.add(HiveJoinAddNotNullRule.INSTANCE_ANTIJOIN);
       rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_JOIN);
       rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_SEMIJOIN);
+      rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_ANTIJOIN);
       rules.add(HiveSortMergeRule.INSTANCE);
       rules.add(HiveSortPullUpConstantsRule.SORT_LIMIT_INSTANCE);
       rules.add(HiveSortPullUpConstantsRule.SORT_EXCHANGE_INSTANCE);
@@ -2130,6 +2134,16 @@ public class CalcitePlanner extends SemanticAnalyzer {
             HiveRemoveSqCountCheck.INSTANCE);
       }
 
+      // 10. Convert left outer join + null filter on right side table column to anti join. Add this
+      // rule after all the optimization for which calcite support for anti join is missing.
+      // Needs to be done before ProjectRemoveRule as it expect a project over filter.
+      // This is done before join re-ordering as join re-ordering is converting the left outer
+      // to right join in some cases before converting back again to left outer.
+      if (conf.getBoolVar(ConfVars.HIVE_CONVERT_ANTI_JOIN)) {
+        generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
+                HiveAntiSemiJoinRule.INSTANCE);
+      }
+
       // Trigger program
       perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
       basePlan = executeProgram(basePlan, program.build(), mdProvider, executorProvider);
@@ -2881,6 +2895,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
         calciteJoinType = JoinRelType.SEMI;
         leftSemiJoin = true;
         break;
+      case ANTI:
+        calciteJoinType = JoinRelType.ANTI;
+        leftSemiJoin = true;
+        break;
       case INNER:
       default:
         calciteJoinType = JoinRelType.INNER;
@@ -2914,10 +2932,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
         final RelDataType combinedRowType = SqlValidatorUtil.createJoinType(
             cluster.getTypeFactory(), inputRels[0].getRowType(), inputRels[1].getRowType(),
             null, ImmutableList.of());
-        topRel = HiveSemiJoin.getSemiJoin(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
-            inputRels[0], inputRels[1],
-            HiveCalciteUtil.fixNullability(cluster.getRexBuilder(),
-                calciteJoinCond, RelOptUtil.getFieldTypeList(combinedRowType)));
+
+        if (hiveJoinType == JoinType.LEFTSEMI) {
+          topRel = HiveSemiJoin.getSemiJoin(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+                  inputRels[0], inputRels[1],
+                  HiveCalciteUtil.fixNullability(cluster.getRexBuilder(),
+                          calciteJoinCond, RelOptUtil.getFieldTypeList(combinedRowType)));
+        } else {
+          topRel = HiveAntiJoin.getAntiJoin(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+                  inputRels[0], inputRels[1],
+                  HiveCalciteUtil.fixNullability(cluster.getRexBuilder(),
+                          calciteJoinCond, RelOptUtil.getFieldTypeList(combinedRowType)));
+        }
 
         // Create join RR: we need to check whether we need to update left RR in case
         // previous call to projectNonColumnEquiConditions updated it
@@ -3019,6 +3045,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
       case HiveParser.TOK_LEFTSEMIJOIN:
         hiveJoinType = JoinType.LEFTSEMI;
         break;
+      case HiveParser.TOK_LEFTANTISEMIJOIN:
+        hiveJoinType = JoinType.ANTI;
+        break;
       default:
         hiveJoinType = JoinType.INNER;
         break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/JoinType.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/JoinType.java
index fd8bb53..18e0de2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/JoinType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/JoinType.java
@@ -23,5 +23,5 @@ package org.apache.hadoop.hive.ql.parse;
  *
  */
 public enum JoinType {
-  INNER, LEFTOUTER, RIGHTOUTER, FULLOUTER, UNIQUE, LEFTSEMI
+  INNER, LEFTOUTER, RIGHTOUTER, FULLOUTER, UNIQUE, LEFTSEMI, ANTI
 };
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 8b1df4b..ce8bf73 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -1478,6 +1478,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         || (node.getToken().getType() == HiveParser.TOK_CROSSJOIN)
         || isOuterJoinToken(node)
         || (node.getToken().getType() == HiveParser.TOK_LEFTSEMIJOIN)
+        || (node.getToken().getType() == HiveParser.TOK_LEFTANTISEMIJOIN)
         || (node.getToken().getType() == HiveParser.TOK_UNIQUEJOIN);
   }
 
@@ -9777,6 +9778,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       joinTree.setNoSemiJoin(false);
       condn[0] = new JoinCond(0, 1, JoinType.LEFTSEMI);
       break;
+    case ANTI:
+      joinTree.setNoSemiJoin(false);
+      condn[0] = new JoinCond(0, 1, JoinType.ANTI);
+      break;
     default:
       condn[0] = new JoinCond(0, 1, JoinType.INNER);
       joinTree.setNoOuterJoin(true);
@@ -9879,6 +9884,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       joinTree.setNoSemiJoin(false);
       condn[0] = new JoinCond(0, 1, JoinType.LEFTSEMI);
       break;
+    case HiveParser.TOK_LEFTANTISEMIJOIN:
+      joinTree.setNoSemiJoin(false);
+      condn[0] = new JoinCond(0, 1, JoinType.ANTI);
+      break;
     default:
       condn[0] = new JoinCond(0, 1, JoinType.INNER);
       joinTree.setNoOuterJoin(true);
@@ -10092,7 +10101,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     for (ASTNode hintNode : hints) {
       for (Node node : hintNode.getChildren()) {
         ASTNode hint = (ASTNode) node;
-        if (hint.getChild(0).getType() != HintParser.TOK_LEFTSEMIJOIN) {
+        if (hint.getChild(0).getType() != HintParser.TOK_LEFTSEMIJOIN &&
+                hint.getChild(0).getType() != HintParser.TOK_LEFTANTISEMIJOIN) {
           continue;
         }
         if (result == null) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinCondDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinCondDesc.java
index 0eb03e9..75f9d81 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinCondDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinCondDesc.java
@@ -73,6 +73,9 @@ public class JoinCondDesc implements Serializable {
     case LEFTSEMI:
       type = JoinDesc.LEFT_SEMI_JOIN;
       break;
+    case ANTI:
+      type = JoinDesc.ANTI_JOIN;
+      break;
     default:
       assert false;
     }
@@ -140,6 +143,9 @@ public class JoinCondDesc implements Serializable {
     case JoinDesc.LEFT_SEMI_JOIN:
       sb.append("Left Semi Join ");
       break;
+    case JoinDesc.ANTI_JOIN:
+      sb.append("Anti Join ");
+      break;
     default:
       sb.append("Unknown Join ");
       break;
@@ -175,6 +181,9 @@ public class JoinCondDesc implements Serializable {
       case JoinDesc.LEFT_SEMI_JOIN:
         join.put("type", "Left Semi");
         break;
+      case JoinDesc.ANTI_JOIN:
+        join.put("type", "Anti");
+        break;
       default:
         join.put("type", "Unknown Join");
         break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
index be2d8e5..9025bc4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
@@ -48,6 +48,7 @@ public class JoinDesc extends AbstractOperatorDesc {
   public static final int FULL_OUTER_JOIN = 3;
   public static final int UNIQUE_JOIN = 4;
   public static final int LEFT_SEMI_JOIN = 5;
+  public static final int ANTI_JOIN = 6;
 
   // used to handle skew join
   private boolean handleSkewJoin = false;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
index a0ee3a9..f32a230 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
@@ -89,7 +89,8 @@ public class VectorMapJoinDesc extends AbstractVectorDesc  {
     INNER_BIG_ONLY,
     LEFT_SEMI,
     OUTER,
-    FULL_OUTER
+    FULL_OUTER,
+    LEFT_ANTI
   }
 
   private HashTableImplementationType hashTableImplementationType;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
index 9666dd7..8be4239 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
@@ -203,6 +203,7 @@ public class PredicateTransitivePropagate extends Transform {
           vector.add(right, left);
           break;
         case JoinDesc.LEFT_OUTER_JOIN:
+        case JoinDesc.ANTI_JOIN:
           vector.add(left, right);
           break;
         case JoinDesc.RIGHT_OUTER_JOIN:
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
index 7316df0..19b7be5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
@@ -339,6 +339,12 @@ public class SyntheticJoinPredicate extends Transform {
           vector.add(right, left);
           break;
         case JoinDesc.LEFT_OUTER_JOIN:
+        case JoinDesc.ANTI_JOIN:
+        //TODO : In case of anti join, bloom filter can be created on left side also ("IN (keylist right table)").
+        // But the filter should be "not-in" ("NOT IN (keylist right table)") as we want to select the records from
+        // left side which are not present in the right side. But it may cause wrong result as
+        // bloom filter may have false positive and thus simply adding not is not correct,
+        // special handling is required for "NOT IN".
           vector.add(right, left);
           break;
         case JoinDesc.RIGHT_OUTER_JOIN:
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestConfig.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestConfig.java
index a250533..37e41743 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestConfig.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestConfig.java
@@ -308,6 +308,9 @@ public class MapJoinTestConfig {
     case LEFT_SEMI:
       joinDescType = JoinDesc.LEFT_SEMI_JOIN;
       break;
+    case LEFT_ANTI:
+        joinDescType = JoinDesc.ANTI_JOIN;
+        break;
     case OUTER:
       joinDescType = JoinDesc.LEFT_OUTER_JOIN;
       break;
@@ -363,6 +366,7 @@ public class MapJoinTestConfig {
       hashTableKind = HashTableKind.HASH_MULTISET;
       break;
     case LEFT_SEMI:
+    case LEFT_ANTI:
       hashTableKind = HashTableKind.HASH_SET;
       break;
     case OUTER:
@@ -1244,4 +1248,4 @@ public class MapJoinTestConfig {
   private static List<Operator<? extends OperatorDesc>> newOperatorList() {
     return new ArrayList<Operator<? extends OperatorDesc>>();
   }
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestData.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestData.java
index 2f5479e..81cc1b6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestData.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestData.java
@@ -281,6 +281,7 @@ public class MapJoinTestData {
     switch (testDesc.vectorMapJoinVariation) {
     case INNER_BIG_ONLY:
     case LEFT_SEMI:
+    case LEFT_ANTI:
       testData.generateRandomSmallTableCounts(testDesc, random);
       break;
     case INNER:
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestMapJoinOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestMapJoinOperator.java
index da74fd5..7e7b356 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestMapJoinOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestMapJoinOperator.java
@@ -160,6 +160,7 @@ public class TestMapJoinOperator {
       return (smallTableValueSize > 0);
     case INNER_BIG_ONLY:
     case LEFT_SEMI:
+    case LEFT_ANTI:
       return (smallTableValueSize == 0);
     case OUTER:
       return true;
@@ -1590,6 +1591,7 @@ public class TestMapJoinOperator {
             break;
           case INNER_BIG_ONLY:
           case LEFT_SEMI:
+          case LEFT_ANTI:
             {
               Object[] outputObjects = new Object[testDesc.outputColumnNames.length];
 
@@ -1792,6 +1794,8 @@ public class TestMapJoinOperator {
     case FULL_OUTER:
       executeTestFullOuter(testDesc, testData, title);
       break;
+    case LEFT_ANTI: //TODO
+      break;
     default:
       throw new RuntimeException("Unexpected Vector MapJoin variation " +
           testDesc.vectorMapJoinVariation);
@@ -2050,4 +2054,4 @@ public class TestMapJoinOperator {
           " variation " + testDesc.vectorMapJoinVariation + option);
     }
   }
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/queries/clientpositive/antijoin.q b/ql/src/test/queries/clientpositive/antijoin.q
new file mode 100644
index 0000000..2b41b25
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/antijoin.q
@@ -0,0 +1,67 @@
+--! qt:dataset:src
+--! qt:dataset:part
+SET hive.vectorized.execution.enabled=false;
+set hive.mapred.mode=nonstrict;
+SET hive.auto.convert.join=false;
+SET hive.auto.convert.anti.join=true;
+-- SORT_QUERY_RESULTS
+
+create table t1_n55 as select cast(key as int) key, value from src where key <= 10;
+
+select * from t1_n55 sort by key;
+
+create table t2_n33 as select cast(2*key as int) key, value from t1_n55;
+
+select * from t2_n33 sort by key;
+
+create table t3_n12 as select * from (select * from t1_n55 union all select * from t2_n33) b;
+select * from t3_n12 sort by key, value;
+
+explain select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value;
+explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value;
+select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value;
+
+explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+
+explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value;
+explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value;
+select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value;
+
+SET hive.vectorized.execution.enabled=true;
+SET hive.auto.convert.join=true;
+explain select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value;
+explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value;
+select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value;
+
+explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+
+explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value;
+explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value;
+select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value;
+
+-- single extra simple filter on right side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and b.value > 'val_1';
+
+-- single extra simple filter on left side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and a.key > 100;
+
+-- single extra non deterministic filter on right side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) > 100;
+
+-- single extra deterministic filter on left side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100;
+
+-- multiple extra deterministic filter on right side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) < 100 and rand(length(b.value)) > 100;
+
+-- multiple extra deterministic filter on left side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100 and rand(length(a.value)) > 100;
+
+-- filter with 'or' are not converted to anti join
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value is null;
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value = 'val_1';
+
diff --git a/ql/src/test/queries/clientpositive/explainuser_1.q b/ql/src/test/queries/clientpositive/explainuser_1.q
index 23bdb79..ce967f4 100644
--- a/ql/src/test/queries/clientpositive/explainuser_1.q
+++ b/ql/src/test/queries/clientpositive/explainuser_1.q
@@ -679,3 +679,46 @@ explain FROM T1_n119 a RIGHT OUTER JOIN T2_n70 c ON c.key+1=a.key select /*+ STR
 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));
 
 explain select /*+ mapjoin(v)*/ sum(hash(k.key)), sum(hash(v.val)) from T1_n119 k left outer join T1_n119 v on k.key+1=v.key;
+
+set hive.auto.convert.anti.join=false;
+
+explain select *
+from src_cbo b
+where not exists
+  (select distinct a.key
+  from src_cbo a
+  where b.value = a.value and a.value > 'val_2'
+  )
+;
+
+explain select *
+from src_cbo b
+group by key, value
+having not exists
+  (select a.key
+  from src_cbo a
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+  )
+;
+
+create view cv1_n5_anti as
+select *
+from src_cbo b
+where not exists
+  (select a.key
+  from src_cbo a
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+;
+
+explain select * from cv1_n5_anti;
+
+explain select *
+from (select *
+      from src_cbo b
+      where not exists
+          (select a.key
+          from src_cbo a
+          where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+     ) a
+;
+set hive.auto.convert.anti.join=true;
\ No newline at end of file
diff --git a/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q b/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q
index 0320107..d277850 100644
--- a/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q
+++ b/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q
@@ -1750,6 +1750,199 @@ order by r, wq, ref, fee
 limit 100;
 
 
+set hive.auto.convert.anti.join=false;
+
+explain
+SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100;
+SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100;
+
+
+
+explain
+SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100;
+SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100;
+set hive.auto.convert.anti.join=true;
+
 DROP TABLE catalog_sales;
 DROP TABLE catalog_returns;
 DROP TABLE store_sales;
diff --git a/ql/src/test/queries/clientpositive/lineage3.q b/ql/src/test/queries/clientpositive/lineage3.q
index e8ef4fe..8d5bd30 100644
--- a/ql/src/test/queries/clientpositive/lineage3.q
+++ b/ql/src/test/queries/clientpositive/lineage3.q
@@ -205,4 +205,11 @@ insert into dest_dp3 partition (y=2, m, d) select first, word, month m, day d wh
 insert into dest_dp2 partition (y=1, m) select f, w, m
 insert into dest_dp1 partition (year=0) select f, w;
 
+set hive.auto.convert.anti.join=false;
+select * from src1 a
+where not exists
+  (select cint from alltypesorc b
+   where a.key = b.ctinyint + 300)
+and key > 300;
+
 reset hive.metastore.disallow.incompatible.col.type.changes;
diff --git a/ql/src/test/queries/clientpositive/subquery_in_having.q b/ql/src/test/queries/clientpositive/subquery_in_having.q
index 732b3e5..9e72184 100644
--- a/ql/src/test/queries/clientpositive/subquery_in_having.q
+++ b/ql/src/test/queries/clientpositive/subquery_in_having.q
@@ -147,6 +147,22 @@ where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value
 group by key, value
 having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key );
 
+set hive.auto.convert.anti.join=false;
+
+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)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key );
+
+
+explain
+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)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key );
+
 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/queries/clientpositive/subquery_multi.q b/ql/src/test/queries/clientpositive/subquery_multi.q
index 03459ac..7389384 100644
--- a/ql/src/test/queries/clientpositive/subquery_multi.q
+++ b/ql/src/test/queries/clientpositive/subquery_multi.q
@@ -143,6 +143,48 @@ select * from part ws1 where
                             and ws1.p_retailprice <> ws2.p_retailprice)
     and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name);
 
+set hive.auto.convert.anti.join=false;
+
+explain select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty);
+select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty);
+
+explain select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty);
+select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty);
+
+
+explain
+select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+ ;
+select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key ) ;
+
+
+explain select count(*)  from src
+    where src.key in (select key from src s1 where s1.key > '9')
+        or src.value is not null
+        or not exists(select key from src);
+
+select count(*)  from src
+    where src.key in (select key from src s1 where s1.key > '9')
+        or src.value is not null
+        or not exists(select key from src);
+
+-- EXISTS and NOT EXISTS with non-equi predicate
+explain select * from part ws1 where
+    exists (select * from part ws2 where ws1.p_type= ws2.p_type
+                            and ws1.p_retailprice <> ws2.p_retailprice)
+    and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name);
+select * from part ws1 where
+    exists (select * from part ws2 where ws1.p_type= ws2.p_type
+                            and ws1.p_retailprice <> ws2.p_retailprice)
+    and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name);
 
 drop table tnull;
 drop table tempty;
diff --git a/ql/src/test/queries/clientpositive/subquery_scalar.q b/ql/src/test/queries/clientpositive/subquery_scalar.q
index 64973a1..598b671 100644
--- a/ql/src/test/queries/clientpositive/subquery_scalar.q
+++ b/ql/src/test/queries/clientpositive/subquery_scalar.q
@@ -148,6 +148,10 @@ select sum(l_extendedprice) from lineitem, part where p_partkey = l_partkey and
 explain select * from part_null_n0 where p_name IN (select p_name from part where part.p_type = part_null_n0.p_type AND p_brand NOT LIKE (select min(p_brand) from part pp where part.p_type = pp.p_type));
 select * from part_null_n0 where p_name IN (select p_name from part where part.p_type = part_null_n0.p_type AND p_brand NOT LIKE (select min(p_brand) from part pp where part.p_type = pp.p_type));
 
+-- non corr, is null , is not converted to anti join.
+explain select * from part where (select i from tnull_n0 limit 1) is null;
+select * from part where (select i from tnull_n0 limit 1) is null;
+
 drop table tnull_n0;
 drop table part_null_n0;
 drop table tempty_n0;
@@ -288,8 +292,6 @@ explain cbo with avg_sales as
        where ss_sold_date_sk = d_date_sk
          and d_year between 1999 and 2001 ) x)
 select * from store_sales where ss_list_price > (select average_sales from avg_sales);
-
-
 DROP TABLE store_sales;
 DROP TABLE date_dim;
 
diff --git a/ql/src/test/results/clientpositive/llap/antijoin.q.out b/ql/src/test/results/clientpositive/llap/antijoin.q.out
new file mode 100644
index 0000000..7b5c288
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/antijoin.q.out
@@ -0,0 +1,1701 @@
+PREHOOK: query: create table t1_n55 as select cast(key as int) key, value from src where key <= 10
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1_n55
+POSTHOOK: query: create table t1_n55 as select cast(key as int) key, value from src where key <= 10
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1_n55
+POSTHOOK: Lineage: t1_n55.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: t1_n55.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select * from t1_n55 sort by key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+#### A masked pattern was here ####
+POSTHOOK: query: select * from t1_n55 sort by key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+#### A masked pattern was here ####
+0	val_0
+0	val_0
+0	val_0
+10	val_10
+2	val_2
+4	val_4
+5	val_5
+5	val_5
+5	val_5
+8	val_8
+9	val_9
+PREHOOK: query: create table t2_n33 as select cast(2*key as int) key, value from t1_n55
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@t1_n55
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2_n33
+POSTHOOK: query: create table t2_n33 as select cast(2*key as int) key, value from t1_n55
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2_n33
+POSTHOOK: Lineage: t2_n33.key EXPRESSION [(t1_n55)t1_n55.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: t2_n33.value SIMPLE [(t1_n55)t1_n55.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: select * from t2_n33 sort by key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select * from t2_n33 sort by key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+0	val_0
+0	val_0
+0	val_0
+10	val_5
+10	val_5
+10	val_5
+16	val_8
+18	val_9
+20	val_10
+4	val_2
+8	val_4
+PREHOOK: query: create table t3_n12 as select * from (select * from t1_n55 union all select * from t2_n33) b
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t3_n12
+POSTHOOK: query: create table t3_n12 as select * from (select * from t1_n55 union all select * from t2_n33) b
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t3_n12
+POSTHOOK: Lineage: t3_n12.key EXPRESSION [(t1_n55)t1_n55.FieldSchema(name:key, type:int, comment:null), (t2_n33)t2_n33.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: t3_n12.value EXPRESSION [(t1_n55)t1_n55.FieldSchema(name:value, type:string, comment:null), (t2_n33)t2_n33.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: select * from t3_n12 sort by key, value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: select * from t3_n12 sort by key, value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+0	val_0
+0	val_0
+0	val_0
+0	val_0
+0	val_0
+0	val_0
+10	val_10
+10	val_5
+10	val_5
+10	val_5
+16	val_8
+18	val_9
+2	val_2
+20	val_10
+4	val_2
+4	val_4
+5	val_5
+5	val_5
+5	val_5
+8	val_4
+8	val_8
+9	val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.5454545
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Anti Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: zz
+                  sort order: ++
+                  Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortExchange(distribution=[any], collation=[[0, 1]])
+  HiveProject(key=[$0], value=[$1])
+    HiveAntiJoin(condition=[=($0, $3)], joinType=[anti])
+      HiveProject(key=[$0], value=[$1])
+        HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+      HiveProject(literalTrue=[true], key=[$0])
+        HiveFilter(condition=[IS NOT NULL($0)])
+          HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+2	val_2
+5	val_5
+5	val_5
+5	val_5
+9	val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.5454545
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Anti Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+  HiveProject(key=[$0], value=[$1])
+    HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+  HiveProject(key=[$0])
+    HiveFilter(condition=[IS NOT NULL($0)])
+      HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+2	val_2
+5	val_5
+5	val_5
+5	val_5
+9	val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.5454545
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Anti Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: zz
+                  sort order: ++
+                  Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortExchange(distribution=[any], collation=[[0, 1]])
+  HiveProject(key=[$1], value=[$2])
+    HiveJoin(condition=[=($1, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveProject(key=[$0])
+        HiveFilter(condition=[IS NOT NULL($0)])
+          HiveTableScan(table=[[default, t3_n12]], table:alias=[c])
+      HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+        HiveProject(key=[$0], value=[$1])
+          HiveFilter(condition=[IS NOT NULL($0)])
+            HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+        HiveProject(key=[$0])
+          HiveFilter(condition=[IS NOT NULL($0)])
+            HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+2	val_2
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+9	val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 3 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Anti Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col0 (type: int)
+                      outputColumnNames: _col0, _col1
+                      input vertices:
+                        1 Map 3
+                      Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: zz
+                        sort order: ++
+                        Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.5454545
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortExchange(distribution=[any], collation=[[0, 1]])
+  HiveProject(key=[$0], value=[$1])
+    HiveAntiJoin(condition=[=($0, $3)], joinType=[anti])
+      HiveProject(key=[$0], value=[$1])
+        HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+      HiveProject(literalTrue=[true], key=[$0])
+        HiveFilter(condition=[IS NOT NULL($0)])
+          HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a where not exists (select 1 from  t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+2	val_2
+5	val_5
+5	val_5
+5	val_5
+9	val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Anti Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col0 (type: int)
+                      outputColumnNames: _col0, _col1
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                        table:
+                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.5454545
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+  HiveProject(key=[$0], value=[$1])
+    HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+  HiveProject(key=[$0])
+    HiveFilter(condition=[IS NOT NULL($0)])
+      HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+2	val_2
+5	val_5
+5	val_5
+5	val_5
+9	val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+        Map 3 <- Map 1 (BROADCAST_EDGE)
+        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Anti Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.5454545
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_46_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.09090909090909091
+                  Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0, _col1
+                        input vertices:
+                          0 Map 1
+                        Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int), _col1 (type: string)
+                          null sort order: zz
+                          sort order: ++
+                          Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortExchange(distribution=[any], collation=[[0, 1]])
+  HiveProject(key=[$1], value=[$2])
+    HiveJoin(condition=[=($1, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+      HiveProject(key=[$0])
+        HiveFilter(condition=[IS NOT NULL($0)])
+          HiveTableScan(table=[[default, t3_n12]], table:alias=[c])
+      HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+        HiveProject(key=[$0], value=[$1])
+          HiveFilter(condition=[IS NOT NULL($0)])
+            HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+        HiveProject(key=[$0])
+          HiveFilter(condition=[IS NOT NULL($0)])
+            HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null  sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+2	val_2
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+5	val_5
+9	val_9
+Warning: Map Join MAPJOIN[15][bigTable=?] in task 'Map 1' is a cross product
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and b.value > 'val_1'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and b.value > 'val_1'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (key is null and key is not null) (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key is null and key is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Limit
+                        Number of rows: 0
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 
+                            1 
+                          input vertices:
+                            1 Map 2
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                          Select Operator
+                            expressions: null (type: int)
+                            outputColumnNames: _col0
+                            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                              table:
+                                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: (key is null and (value > 'val_1') and key is not null) (type: boolean)
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key is null and (value > 'val_1') and key is not null) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and a.key > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and a.key > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: (key > 100) (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key > 100) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Anti Join 0 to 1
+                        keys:
+                          0 _col0 (type: int)
+                          1 _col0 (type: int)
+                        outputColumnNames: _col0
+                        input vertices:
+                          1 Map 2
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          table:
+                              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: (key > 100) (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (key > 100) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (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)
+                          null sort order: z
+                          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: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Left Outer Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col0 (type: int)
+                      outputColumnNames: _col0, _col1, _col2
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 17 Data size: 1649 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: (_col1 is null and (rand(length(_col2)) > 100.0D)) (type: boolean)
+                        Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                        Select Operator
+                          expressions: _col0 (type: int)
+                          outputColumnNames: _col0
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                            table:
+                                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Anti Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col0 (type: int)
+                      outputColumnNames: _col0
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: (rand(_col0) > 100.0D) (type: boolean)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        File Output Operator
+                          compressed: false
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          table:
+                              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.5454545
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) < 100 and rand(length(b.value)) > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) < 100 and rand(length(b.value)) > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Left Outer Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col0 (type: int)
+                      outputColumnNames: _col0, _col1, _col2
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 17 Data size: 1649 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: (_col1 is null and (rand(length(_col2)) < 100.0D) and (rand(length(_col2)) > 100.0D)) (type: boolean)
+                        Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                        Select Operator
+                          expressions: _col0 (type: int)
+                          outputColumnNames: _col0
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                            table:
+                                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100 and rand(length(a.value)) > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100 and rand(length(a.value)) > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Anti Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col0 (type: int)
+                      outputColumnNames: _col0, _col1
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: ((rand(_col0) > 100.0D) and (rand(length(_col1)) > 100.0D)) (type: boolean)
+                        Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+                        Select Operator
+                          expressions: _col0 (type: int)
+                          outputColumnNames: _col0
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                            table:
+                                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int)
+                        minReductionHashAggr: 0.5454545
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: int)
+                          Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Left Outer Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col0 (type: int)
+                      outputColumnNames: _col0, _col1, _col2
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 17 Data size: 1649 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: (_col1 is null or _col2 is null) (type: boolean)
+                        Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+                        Select Operator
+                          expressions: _col0 (type: int)
+                          outputColumnNames: _col0
+                          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                            table:
+                                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value = 'val_1'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value = 'val_1'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map Join Operator
+                      condition map:
+                           Left Outer Join 0 to 1
+                      keys:
+                        0 _col0 (type: int)
+                        1 _col0 (type: int)
+                      outputColumnNames: _col0, _col1, _col2
+                      input vertices:
+                        1 Map 2
+                      Statistics: Num rows: 17 Data size: 1649 Basic stats: COMPLETE Column stats: COMPLETE
+                      Filter Operator
+                        predicate: (_col1 is null or (_col2 = 'val_1')) (type: boolean)
+                        Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE
+                        Select Operator
+                          expressions: _col0 (type: int)
+                          outputColumnNames: _col0
+                          Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                            table:
+                                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: key is not null (type: boolean)
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: key is not null (type: boolean)
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: int), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
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 548fc65..cce2f97 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -2092,39 +2092,33 @@ Stage-0
     limit:-1
     Stage-1
       Reducer 2 llap
-      File Output Operator [FS_19]
-        Select Operator [SEL_18] (rows=500 width=178)
-          Output:["_col0","_col1"]
-          Filter Operator [FIL_17] (rows=500 width=178)
-            predicate:_col2 is null
-            Merge Join Operator [MERGEJOIN_27] (rows=566 width=178)
-              Conds:RS_14._col1=RS_15._col1(Left Outer),Output:["_col0","_col1","_col2"]
-            <-Map 1 [SIMPLE_EDGE] llap
-              SHUFFLE [RS_14]
-                PartitionCols:_col1
-                Select Operator [SEL_1] (rows=500 width=178)
-                  Output:["_col0","_col1"]
-                  TableScan [TS_0] (rows=500 width=178)
-                    default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-            <-Reducer 3 [SIMPLE_EDGE] llap
-              SHUFFLE [RS_15]
-                PartitionCols:_col1
-                Select Operator [SEL_13] (rows=41 width=95)
-                  Output:["_col0","_col1"]
-                  Group By Operator [GBY_12] (rows=41 width=91)
-                    Output:["_col0"],keys:_col1
-                    Select Operator [SEL_8] (rows=83 width=178)
-                      Output:["_col1"]
-                      Group By Operator [GBY_7] (rows=83 width=178)
-                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                      <-Map 1 [SIMPLE_EDGE] llap
-                        SHUFFLE [RS_6]
-                          PartitionCols:_col0
-                          Group By Operator [GBY_5] (rows=83 width=178)
-                            Output:["_col0","_col1"],keys:value, key
-                            Filter Operator [FIL_21] (rows=166 width=178)
-                              predicate:(value > 'val_2')
-                               Please refer to the previous TableScan [TS_0]
+      File Output Operator [FS_15]
+        Merge Join Operator [MERGEJOIN_23] (rows=434 width=178)
+          Conds:RS_11._col1=RS_12._col0(Anti),Output:["_col0","_col1"]
+        <-Map 1 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_11]
+            PartitionCols:_col1
+            Select Operator [SEL_1] (rows=500 width=178)
+              Output:["_col0","_col1"]
+              TableScan [TS_0] (rows=500 width=178)
+                default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+        <-Reducer 3 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_12]
+            PartitionCols:_col0
+            Group By Operator [GBY_10] (rows=41 width=91)
+              Output:["_col0"],keys:_col0
+              Select Operator [SEL_8] (rows=83 width=91)
+                Output:["_col0"]
+                Group By Operator [GBY_7] (rows=83 width=178)
+                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                <-Map 1 [SIMPLE_EDGE] llap
+                  SHUFFLE [RS_6]
+                    PartitionCols:_col0, _col1
+                    Group By Operator [GBY_5] (rows=83 width=178)
+                      Output:["_col0","_col1"],keys:value, key
+                      Filter Operator [FIL_17] (rows=166 width=178)
+                        predicate:(value > 'val_2')
+                         Please refer to the previous TableScan [TS_0]
 
 PREHOOK: query: explain select * 
 from src_cbo b 
@@ -2151,45 +2145,41 @@ POSTHOOK: Input: default@src_cbo
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:-1
     Stage-1
-      Reducer 2 llap
-      File Output Operator [FS_18]
-        Select Operator [SEL_17] (rows=250 width=178)
-          Output:["_col0","_col1"]
-          Filter Operator [FIL_16] (rows=250 width=179)
-            predicate:_col2 is null
-            Merge Join Operator [MERGEJOIN_31] (rows=333 width=179)
-              Conds:GBY_4._col0, _col1=DUMMY_STORE_32._col1, _col2(Left Outer),Output:["_col0","_col1","_col2"]
-            <-Dummy Store [DUMMY_STORE_32]
-                Select Operator [SEL_12] (rows=83 width=182)
-                  Output:["_col0","_col1","_col2"]
-                  Group By Operator [GBY_11] (rows=83 width=178)
-                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-            <-Group By Operator [GBY_4] (rows=250 width=178)
-                Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-              <-Map 1 [SIMPLE_EDGE] llap
-                SHUFFLE [RS_3]
-                  PartitionCols:_col0, _col1
-                  Group By Operator [GBY_2] (rows=250 width=178)
-                    Output:["_col0","_col1"],keys:key, value
-                    Select Operator [SEL_1] (rows=500 width=178)
-                      Output:["key","value"]
-                      TableScan [TS_0] (rows=500 width=178)
-                        default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-              <-Map 3 [SIMPLE_EDGE] llap
-                SHUFFLE [RS_10]
-                  PartitionCols:_col0, _col1
-                  Group By Operator [GBY_9] (rows=83 width=178)
-                    Output:["_col0","_col1"],keys:key, value
-                    Filter Operator [FIL_20] (rows=166 width=178)
-                      predicate:((value > 'val_12') and key is not null)
-                      TableScan [TS_6] (rows=500 width=178)
-                        default@src_cbo,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+      Reducer 3 llap
+      File Output Operator [FS_15]
+        Merge Join Operator [MERGEJOIN_28] (rows=167 width=178)
+          Conds:RS_11._col0, _col1=RS_12._col0, _col1(Anti),Output:["_col0","_col1"]
+        <-Map 1 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_12]
+            PartitionCols:_col0, _col1
+            Group By Operator [GBY_10] (rows=83 width=178)
+              Output:["_col0","_col1"],keys:_col0, _col1
+              Select Operator [SEL_8] (rows=166 width=178)
+                Output:["_col0","_col1"]
+                Filter Operator [FIL_17] (rows=166 width=178)
+                  predicate:((value > 'val_12') and key is not null)
+                  TableScan [TS_0] (rows=500 width=178)
+                    default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+        <-Reducer 2 [SIMPLE_EDGE] llap
+          SHUFFLE [RS_11]
+            PartitionCols:_col0, _col1
+            Group By Operator [GBY_4] (rows=250 width=178)
+              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+            <-Map 1 [SIMPLE_EDGE] llap
+              SHUFFLE [RS_3]
+                PartitionCols:_col0, _col1
+                Group By Operator [GBY_2] (rows=250 width=178)
+                  Output:["_col0","_col1"],keys:key, value
+                  Select Operator [SEL_1] (rows=500 width=178)
+                    Output:["key","value"]
+                     Please refer to the previous TableScan [TS_0]
 
 PREHOOK: query: create view cv1_n5 as 
 select * 
@@ -6695,3 +6685,297 @@ Stage-0
                       TableScan [TS_0] (rows=1 width=184)
                         default@t1_n119,k,Tbl:COMPLETE,Col:NONE,Output:["key"]
 
+PREHOOK: query: explain select *
+from src_cbo b
+where not exists
+  (select distinct a.key
+  from src_cbo a
+  where b.value = a.value and a.value > 'val_2'
+  )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+POSTHOOK: query: explain select *
+from src_cbo b
+where not exists
+  (select distinct a.key
+  from src_cbo a
+  where b.value = a.value and a.value > 'val_2'
+  )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Map 1 <- Reducer 6 (BROADCAST_EDGE)
+Reducer 3 <- Map 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Map 1 llap
+      File Output Operator [FS_21]
+        Select Operator [SEL_20] (rows=500 width=178)
+          Output:["_col0","_col1"]
+          Filter Operator [FIL_19] (rows=500 width=179)
+            predicate:_col2 is null
+            Map Join Operator [MAPJOIN_29] (rows=666 width=179)
+              Conds:SEL_1._col1=RS_17._col1(Left Outer),Output:["_col0","_col1","_col2"]
+            <-Reducer 6 [BROADCAST_EDGE] llap
+              BROADCAST [RS_17]
+                PartitionCols:_col1
+                Select Operator [SEL_15] (rows=102 width=95)
+                  Output:["_col0","_col1"]
+                  Group By Operator [GBY_14] (rows=102 width=91)
+                    Output:["_col0"],keys:KEY._col0
+                  <-Reducer 5 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_13]
+                      PartitionCols:_col0
+                      Group By Operator [GBY_12] (rows=166 width=91)
+                        Output:["_col0"],keys:KEY._col0
+                      <-Reducer 4 [SIMPLE_EDGE] llap
+                        SHUFFLE [RS_11]
+                          PartitionCols:rand()
+                          Select Operator [SEL_9] (rows=166 width=178)
+                            Output:["_col1"]
+                            Group By Operator [GBY_8] (rows=166 width=178)
+                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                            <-Reducer 3 [SIMPLE_EDGE] llap
+                              SHUFFLE [RS_7]
+                                PartitionCols:_col0, _col1
+                                Group By Operator [GBY_6] (rows=166 width=178)
+                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                                <-Map 2 [SIMPLE_EDGE] llap
+                                  SHUFFLE [RS_5]
+                                    PartitionCols:rand()
+                                    Filter Operator [FIL_23] (rows=166 width=178)
+                                      predicate:(value > 'val_2')
+                                      TableScan [TS_2] (rows=500 width=178)
+                                        default@src_cbo,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+            <-Select Operator [SEL_1] (rows=500 width=178)
+                Output:["_col0","_col1"]
+                TableScan [TS_0] (rows=500 width=178)
+                  default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+
+PREHOOK: query: explain select *
+from src_cbo b
+group by key, value
+having not exists
+  (select a.key
+  from src_cbo a
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+  )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+POSTHOOK: query: explain select *
+from src_cbo b
+group by key, value
+having not exists
+  (select a.key
+  from src_cbo a
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+  )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (BROADCAST_EDGE)
+Reducer 4 <- Map 1 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Reducer 3 llap
+      File Output Operator [FS_20]
+        Select Operator [SEL_19] (rows=500 width=178)
+          Output:["_col0","_col1"]
+          Filter Operator [FIL_18] (rows=500 width=179)
+            predicate:_col2 is null
+            Map Join Operator [MAPJOIN_33] (rows=762 width=179)
+              Conds:GBY_5._col0, _col1=RS_16._col1, _col2(Left Outer),Output:["_col0","_col1","_col2"]
+            <-Reducer 5 [BROADCAST_EDGE] llap
+              BROADCAST [RS_16]
+                PartitionCols:_col1, _col2
+                Select Operator [SEL_14] (rows=166 width=182)
+                  Output:["_col0","_col1","_col2"]
+                  Group By Operator [GBY_13] (rows=166 width=178)
+                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                  <-Reducer 4 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_12]
+                      PartitionCols:_col0, _col1
+                      Group By Operator [GBY_11] (rows=166 width=178)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Map 1 [SIMPLE_EDGE] llap
+                        SHUFFLE [RS_10]
+                          PartitionCols:rand()
+                          Filter Operator [FIL_22] (rows=166 width=178)
+                            predicate:((value > 'val_12') and key is not null)
+                            TableScan [TS_0] (rows=500 width=178)
+                              default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+            <-Group By Operator [GBY_5] (rows=500 width=178)
+                Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+              <-Reducer 2 [SIMPLE_EDGE] llap
+                SHUFFLE [RS_4]
+                  PartitionCols:_col0, _col1
+                  Group By Operator [GBY_3] (rows=500 width=178)
+                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                  <-Map 1 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_2]
+                      PartitionCols:rand()
+                      Select Operator [SEL_1] (rows=500 width=178)
+                        Output:["key","value"]
+                         Please refer to the previous TableScan [TS_0]
+
+PREHOOK: query: create view cv1_n5_anti as
+select *
+from src_cbo b
+where not exists
+  (select a.key
+  from src_cbo a
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@src_cbo
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cv1_n5_anti
+POSTHOOK: query: create view cv1_n5_anti as
+select *
+from src_cbo b
+where not exists
+  (select a.key
+  from src_cbo a
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@src_cbo
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cv1_n5_anti
+POSTHOOK: Lineage: cv1_n5_anti.key SIMPLE [(src_cbo)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: cv1_n5_anti.value SIMPLE [(src_cbo)b.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: explain select * from cv1_n5_anti
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cv1_n5_anti
+PREHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from cv1_n5_anti
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cv1_n5_anti
+POSTHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Map 1 <- Reducer 4 (BROADCAST_EDGE)
+Reducer 3 <- Map 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Map 1 llap
+      File Output Operator [FS_15]
+        Select Operator [SEL_14] (rows=500 width=178)
+          Output:["_col0","_col1"]
+          Filter Operator [FIL_13] (rows=500 width=179)
+            predicate:_col2 is null
+            Map Join Operator [MAPJOIN_28] (rows=762 width=179)
+              Conds:SEL_1._col0, _col1=RS_11._col1, _col2(Left Outer),Output:["_col0","_col1","_col2"]
+            <-Reducer 4 [BROADCAST_EDGE] llap
+              BROADCAST [RS_11]
+                PartitionCols:_col1, _col2
+                Select Operator [SEL_9] (rows=166 width=182)
+                  Output:["_col0","_col1","_col2"]
+                  Group By Operator [GBY_8] (rows=166 width=178)
+                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                  <-Reducer 3 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_7]
+                      PartitionCols:_col0, _col1
+                      Group By Operator [GBY_6] (rows=166 width=178)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Map 2 [SIMPLE_EDGE] llap
+                        SHUFFLE [RS_5]
+                          PartitionCols:rand()
+                          Filter Operator [FIL_17] (rows=166 width=178)
+                            predicate:((value > 'val_9') and key is not null)
+                            TableScan [TS_2] (rows=500 width=178)
+                              default@src_cbo,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"],properties:{"insideView":"TRUE"}
+            <-Select Operator [SEL_1] (rows=500 width=178)
+                Output:["_col0","_col1"]
+                TableScan [TS_0] (rows=500 width=178)
+                  default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"],properties:{"insideView":"TRUE"}
+
+PREHOOK: query: explain select *
+from (select *
+      from src_cbo b
+      where not exists
+          (select a.key
+          from src_cbo a
+          where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+     ) a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+POSTHOOK: query: explain select *
+from (select *
+      from src_cbo b
+      where not exists
+          (select a.key
+          from src_cbo a
+          where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+     ) a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Map 1 <- Reducer 4 (BROADCAST_EDGE)
+Reducer 3 <- Map 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+
+Stage-0
+  Fetch Operator
+    limit:-1
+    Stage-1
+      Map 1 llap
+      File Output Operator [FS_15]
+        Select Operator [SEL_14] (rows=500 width=178)
+          Output:["_col0","_col1"]
+          Filter Operator [FIL_13] (rows=500 width=179)
+            predicate:_col2 is null
+            Map Join Operator [MAPJOIN_28] (rows=762 width=179)
+              Conds:SEL_1._col0, _col1=RS_11._col1, _col2(Left Outer),Output:["_col0","_col1","_col2"]
+            <-Reducer 4 [BROADCAST_EDGE] llap
+              BROADCAST [RS_11]
+                PartitionCols:_col1, _col2
+                Select Operator [SEL_9] (rows=166 width=182)
+                  Output:["_col0","_col1","_col2"]
+                  Group By Operator [GBY_8] (rows=166 width=178)
+                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                  <-Reducer 3 [SIMPLE_EDGE] llap
+                    SHUFFLE [RS_7]
+                      PartitionCols:_col0, _col1
+                      Group By Operator [GBY_6] (rows=166 width=178)
+                        Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+                      <-Map 2 [SIMPLE_EDGE] llap
+                        SHUFFLE [RS_5]
+                          PartitionCols:rand()
+                          Filter Operator [FIL_17] (rows=166 width=178)
+                            predicate:((value > 'val_9') and key is not null)
+                            TableScan [TS_2] (rows=500 width=178)
+                              default@src_cbo,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+            <-Select Operator [SEL_1] (rows=500 width=178)
+                Output:["_col0","_col1"]
+                TableScan [TS_0] (rows=500 width=178)
+                  default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+
diff --git a/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out b/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out
index b7d3c9a..dd7581f 100644
--- a/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out
+++ b/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out
@@ -1811,7 +1811,6 @@ WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t
 INNER JOIN (SELECT "d_date_sk"
 FROM "DATE_DIM"
 WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
-GROUP BY "t0"."ws_bill_customer_sk"
                     hive.sql.query.fieldNames literalTrue,ws_bill_customer_sk0
                     hive.sql.query.fieldTypes boolean,int
                     hive.sql.query.split false
@@ -1841,22 +1840,26 @@ WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t
 INNER JOIN (SELECT "d_date_sk"
 FROM "DATE_DIM"
 WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
-GROUP BY "t0"."cs_ship_customer_sk"
                     hive.sql.query.fieldNames literalTrue,cs_ship_customer_sk0
                     hive.sql.query.fieldTypes boolean,int
                     hive.sql.query.split false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: literaltrue (type: boolean), cs_ship_customer_sk0 (type: int)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col0 (type: boolean)
+                    expressions: cs_ship_customer_sk0 (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        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
         Reducer 2 
@@ -1907,39 +1910,32 @@ GROUP BY "t0"."cs_ship_customer_sk"
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Anti Join 0 to 1
                 keys:
                   0 _col0 (type: int)
-                  1 _col1 (type: int)
-                outputColumnNames: _col32, _col33, _col34, _col35, _col36, _col42
+                  1 _col0 (type: int)
+                outputColumnNames: _col32, _col33, _col34, _col35, _col36
                 Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                Filter Operator
-                  predicate: _col42 is null (type: boolean)
+                Top N Key Operator
+                  sort order: +++++
+                  keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                  null sort order: zzzzz
                   Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                  Top N Key Operator
-                    sort order: +++++
+                  top n: 100
+                  Group By Operator
+                    aggregations: count()
                     keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
-                    null sort order: zzzzz
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                     Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                    top n: 100
-                    Select Operator
-                      expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
-                      outputColumnNames: _col32, _col33, _col34, _col35, _col36
+                    Reduce Output Operator
+                      key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                      null sort order: zzzzz
+                      sort order: +++++
+                      Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
                       Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: count()
-                        keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
-                        minReductionHashAggr: 0.99
-                        mode: hash
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
-                          null sort order: zzzzz
-                          sort order: +++++
-                          Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
-                          Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col5 (type: bigint)
+                      value expressions: _col5 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -2299,7 +2295,6 @@ WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t
 INNER JOIN (SELECT "d_date_sk"
 FROM "DATE_DIM"
 WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
-GROUP BY "t0"."ws_bill_customer_sk"
                     hive.sql.query.fieldNames literalTrue,ws_bill_customer_sk0
                     hive.sql.query.fieldTypes boolean,int
                     hive.sql.query.split false
@@ -2329,22 +2324,26 @@ WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t
 INNER JOIN (SELECT "d_date_sk"
 FROM "DATE_DIM"
 WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
-GROUP BY "t0"."cs_ship_customer_sk"
                     hive.sql.query.fieldNames literalTrue,cs_ship_customer_sk0
                     hive.sql.query.fieldTypes boolean,int
                     hive.sql.query.split false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    expressions: literaltrue (type: boolean), cs_ship_customer_sk0 (type: int)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                    Reduce Output Operator
-                      key expressions: _col1 (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col1 (type: int)
-                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
-                      value expressions: _col0 (type: boolean)
+                    expressions: cs_ship_customer_sk0 (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        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
         Reducer 2 
@@ -2395,39 +2394,32 @@ GROUP BY "t0"."cs_ship_customer_sk"
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Anti Join 0 to 1
                 keys:
                   0 _col0 (type: int)
-                  1 _col1 (type: int)
-                outputColumnNames: _col32, _col33, _col34, _col35, _col36, _col42
+                  1 _col0 (type: int)
+                outputColumnNames: _col32, _col33, _col34, _col35, _col36
                 Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                Filter Operator
-                  predicate: _col42 is null (type: boolean)
+                Top N Key Operator
+                  sort order: +++++
+                  keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                  null sort order: zzzzz
                   Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                  Top N Key Operator
-                    sort order: +++++
+                  top n: 100
+                  Group By Operator
+                    aggregations: count()
                     keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
-                    null sort order: zzzzz
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                     Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                    top n: 100
-                    Select Operator
-                      expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
-                      outputColumnNames: _col32, _col33, _col34, _col35, _col36
+                    Reduce Output Operator
+                      key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                      null sort order: zzzzz
+                      sort order: +++++
+                      Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
                       Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                      Group By Operator
-                        aggregations: count()
-                        keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
-                        minReductionHashAggr: 0.99
-                        mode: hash
-                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                        Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
-                          null sort order: zzzzz
-                          sort order: +++++
-                          Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
-                          Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
-                          value expressions: _col5 (type: bigint)
+                      value expressions: _col5 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -5818,6 +5810,974 @@ POSTHOOK: Input: default@reason
 POSTHOOK: Input: default@web_returns
 POSTHOOK: Input: default@web_sales
 #### A masked pattern was here ####
+PREHOOK: query: explain
+SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  properties:
+                    hive.sql.query SELECT "t0"."c_customer_sk", "t0"."c_customer_id", "t0"."c_current_cdemo_sk", "t0"."c_current_hdemo_sk", "t0"."c_current_addr_sk", "t0"."c_first_shipto_date_sk", "t0"."c_first_sales_date_sk", "t0"."c_salutation", "t0"."c_first_name", "t0"."c_last_name", "t0"."c_preferred_cust_flag", "t0"."c_birth_day", "t0"."c_birth_month", "t0"."c_birth_year", "t0"."c_birth_country", "t0"."c_login", "t0"."c_email_address", "t0"."c_last_review_date", "t4"."ca_address_sk [...]
+FROM (SELECT "c_customer_sk", "c_customer_id", "c_current_cdemo_sk", "c_current_hdemo_sk", "c_current_addr_sk", "c_first_shipto_date_sk", "c_first_sales_date_sk", "c_salutation", "c_first_name", "c_last_name", "c_preferred_cust_flag", "c_birth_day", "c_birth_month", "c_birth_year", "c_birth_country", "c_login", "c_email_address", "c_last_review_date"
+FROM "CUSTOMER"
+WHERE "c_current_addr_sk" IS NOT NULL AND "c_current_cdemo_sk" IS NOT NULL AND "c_customer_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "cd_demo_sk", "cd_gender", "cd_marital_status", "cd_education_status", "cd_purchase_estimate", "cd_credit_rating", "cd_dep_count", "cd_dep_employed_count", "cd_dep_college_count"
+FROM "CUSTOMER_DEMOGRAPHICS"
+WHERE "cd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."c_current_cdemo_sk" = "t2"."cd_demo_sk"
+INNER JOIN (SELECT "ca_address_sk", "ca_address_id", "ca_street_number", "ca_street_name", "ca_street_type", "ca_suite_number", "ca_city", "ca_county", "ca_state", "ca_zip", "ca_country", "ca_gmt_offset", "ca_location_type"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_state" IN ('CO', 'IL', 'MN') AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."c_current_addr_sk" = "t4"."ca_address_sk"
+                    hive.sql.query.fieldNames c_customer_sk,c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk,ca_address_id,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_locati [...]
+                    hive.sql.query.fieldTypes int,char(16),int,int,int,int,int,char(10),char(20),char(30),char(1),int,int,int,varchar(20),char(13),char(50),char(10),int,char(16),char(10),varchar(60),char(15),char(10),varchar(60),varchar(30),char(2),char(10),varchar(20),decimal(5,2),char(20),int,char(1),char(1),char(20),int,char(10),int,int,int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: c_customer_sk (type: int), cd_gender (type: char(1)), cd_marital_status (type: char(1)), cd_education_status (type: char(20)), cd_purchase_estimate (type: int), cd_credit_rating (type: char(10))
+                    outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ss_customer_sk" AS "ss_customer_sk0"
+FROM (SELECT "ss_sold_date_sk", "ss_sold_time_sk", "ss_item_sk", "ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk", "ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity", "ss_wholesale_cost", "ss_list_price", "ss_sales_price", "ss_ext_discount_amt", "ss_ext_sales_price", "ss_ext_wholesale_cost", "ss_ext_list_price", "ss_ext_tax", "ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax", "ss_net_profit"
+FROM "STORE_SALES"
+WHERE "ss_customer_sk" IS NOT NULL AND "ss_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ss_sold_date_sk" = "t2"."d_date_sk"
+                    hive.sql.query.fieldNames ss_customer_sk0
+                    hive.sql.query.fieldTypes int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ss_customer_sk0 (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        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 8 
+            Map Operator Tree:
+                TableScan
+                  alias: web_sales
+                  properties:
+                    hive.sql.query SELECT TRUE AS "literalTrue", "t0"."ws_bill_customer_sk" AS "ws_bill_customer_sk0"
+FROM (SELECT "ws_sold_date_sk", "ws_sold_time_sk", "ws_ship_date_sk", "ws_item_sk", "ws_bill_customer_sk", "ws_bill_cdemo_sk", "ws_bill_hdemo_sk", "ws_bill_addr_sk", "ws_ship_customer_sk", "ws_ship_cdemo_sk", "ws_ship_hdemo_sk", "ws_ship_addr_sk", "ws_web_page_sk", "ws_web_site_sk", "ws_ship_mode_sk", "ws_warehouse_sk", "ws_promo_sk", "ws_order_number", "ws_quantity", "ws_wholesale_cost", "ws_list_price", "ws_sales_price", "ws_ext_discount_amt", "ws_ext_sales_price", "ws_ext_wholesale_co [...]
+FROM "WEB_SALES"
+WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
+                    hive.sql.query.fieldNames literalTrue,ws_bill_customer_sk0
+                    hive.sql.query.fieldTypes boolean,int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: literaltrue (type: boolean), ws_bill_customer_sk0 (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col1 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: boolean)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: catalog_sales
+                  properties:
+                    hive.sql.query SELECT TRUE AS "literalTrue", "t0"."cs_ship_customer_sk" AS "cs_ship_customer_sk0"
+FROM (SELECT "cs_sold_date_sk", "cs_sold_time_sk", "cs_ship_date_sk", "cs_bill_customer_sk", "cs_bill_cdemo_sk", "cs_bill_hdemo_sk", "cs_bill_addr_sk", "cs_ship_customer_sk", "cs_ship_cdemo_sk", "cs_ship_hdemo_sk", "cs_ship_addr_sk", "cs_call_center_sk", "cs_catalog_page_sk", "cs_ship_mode_sk", "cs_warehouse_sk", "cs_item_sk", "cs_promo_sk", "cs_order_number", "cs_quantity", "cs_wholesale_cost", "cs_list_price", "cs_sales_price", "cs_ext_discount_amt", "cs_ext_sales_price", "cs_ext_whole [...]
+FROM "CATALOG_SALES"
+WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
+                    hive.sql.query.fieldNames literalTrue,cs_ship_customer_sk0
+                    hive.sql.query.fieldTypes boolean,int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: cs_ship_customer_sk0 (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        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
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36, _col40
+                Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: _col40 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: int), _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                    outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Anti Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col32, _col33, _col34, _col35, _col36
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Top N Key Operator
+                  sort order: +++++
+                  keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                  null sort order: zzzzz
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  top n: 100
+                  Group By Operator
+                    aggregations: count()
+                    keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                      null sort order: zzzzz
+                      sort order: +++++
+                      Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                      Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col5 (type: bigint)
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: char(1)), KEY._col1 (type: char(1)), KEY._col2 (type: char(20)), KEY._col3 (type: int), KEY._col4 (type: char(10))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col5 (type: bigint), _col3 (type: int), _col4 (type: char(10))
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col4 (type: int), _col6 (type: char(10))
+                    null sort order: zzzzz
+                    sort order: +++++
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col3 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: char(1)), KEY.reducesinkkey1 (type: char(1)), KEY.reducesinkkey2 (type: char(20)), VALUE._col0 (type: bigint), KEY.reducesinkkey3 (type: int), VALUE._col0 (type: bigint), KEY.reducesinkkey4 (type: char(10)), VALUE._col0 (type: bigint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+PREHOOK: query: explain
+SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  properties:
+                    hive.sql.query SELECT "t0"."c_customer_sk", "t0"."c_customer_id", "t0"."c_current_cdemo_sk", "t0"."c_current_hdemo_sk", "t0"."c_current_addr_sk", "t0"."c_first_shipto_date_sk", "t0"."c_first_sales_date_sk", "t0"."c_salutation", "t0"."c_first_name", "t0"."c_last_name", "t0"."c_preferred_cust_flag", "t0"."c_birth_day", "t0"."c_birth_month", "t0"."c_birth_year", "t0"."c_birth_country", "t0"."c_login", "t0"."c_email_address", "t0"."c_last_review_date", "t4"."ca_address_sk [...]
+FROM (SELECT "c_customer_sk", "c_customer_id", "c_current_cdemo_sk", "c_current_hdemo_sk", "c_current_addr_sk", "c_first_shipto_date_sk", "c_first_sales_date_sk", "c_salutation", "c_first_name", "c_last_name", "c_preferred_cust_flag", "c_birth_day", "c_birth_month", "c_birth_year", "c_birth_country", "c_login", "c_email_address", "c_last_review_date"
+FROM "CUSTOMER"
+WHERE "c_current_addr_sk" IS NOT NULL AND "c_current_cdemo_sk" IS NOT NULL AND "c_customer_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "cd_demo_sk", "cd_gender", "cd_marital_status", "cd_education_status", "cd_purchase_estimate", "cd_credit_rating", "cd_dep_count", "cd_dep_employed_count", "cd_dep_college_count"
+FROM "CUSTOMER_DEMOGRAPHICS"
+WHERE "cd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."c_current_cdemo_sk" = "t2"."cd_demo_sk"
+INNER JOIN (SELECT "ca_address_sk", "ca_address_id", "ca_street_number", "ca_street_name", "ca_street_type", "ca_suite_number", "ca_city", "ca_county", "ca_state", "ca_zip", "ca_country", "ca_gmt_offset", "ca_location_type"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_state" IN ('CO', 'IL', 'MN') AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."c_current_addr_sk" = "t4"."ca_address_sk"
+                    hive.sql.query.fieldNames c_customer_sk,c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk,ca_address_id,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_locati [...]
+                    hive.sql.query.fieldTypes int,char(16),int,int,int,int,int,char(10),char(20),char(30),char(1),int,int,int,varchar(20),char(13),char(50),char(10),int,char(16),char(10),varchar(60),char(15),char(10),varchar(60),varchar(30),char(2),char(10),varchar(20),decimal(5,2),char(20),int,char(1),char(1),char(20),int,char(10),int,int,int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: c_customer_sk (type: int), cd_gender (type: char(1)), cd_marital_status (type: char(1)), cd_education_status (type: char(20)), cd_purchase_estimate (type: int), cd_credit_rating (type: char(10))
+                    outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ss_customer_sk" AS "ss_customer_sk0"
+FROM (SELECT "ss_sold_date_sk", "ss_sold_time_sk", "ss_item_sk", "ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk", "ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity", "ss_wholesale_cost", "ss_list_price", "ss_sales_price", "ss_ext_discount_amt", "ss_ext_sales_price", "ss_ext_wholesale_cost", "ss_ext_list_price", "ss_ext_tax", "ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax", "ss_net_profit"
+FROM "STORE_SALES"
+WHERE "ss_customer_sk" IS NOT NULL AND "ss_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ss_sold_date_sk" = "t2"."d_date_sk"
+                    hive.sql.query.fieldNames ss_customer_sk0
+                    hive.sql.query.fieldTypes int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ss_customer_sk0 (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        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 8 
+            Map Operator Tree:
+                TableScan
+                  alias: web_sales
+                  properties:
+                    hive.sql.query SELECT TRUE AS "literalTrue", "t0"."ws_bill_customer_sk" AS "ws_bill_customer_sk0"
+FROM (SELECT "ws_sold_date_sk", "ws_sold_time_sk", "ws_ship_date_sk", "ws_item_sk", "ws_bill_customer_sk", "ws_bill_cdemo_sk", "ws_bill_hdemo_sk", "ws_bill_addr_sk", "ws_ship_customer_sk", "ws_ship_cdemo_sk", "ws_ship_hdemo_sk", "ws_ship_addr_sk", "ws_web_page_sk", "ws_web_site_sk", "ws_ship_mode_sk", "ws_warehouse_sk", "ws_promo_sk", "ws_order_number", "ws_quantity", "ws_wholesale_cost", "ws_list_price", "ws_sales_price", "ws_ext_discount_amt", "ws_ext_sales_price", "ws_ext_wholesale_co [...]
+FROM "WEB_SALES"
+WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
+                    hive.sql.query.fieldNames literalTrue,ws_bill_customer_sk0
+                    hive.sql.query.fieldTypes boolean,int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: literaltrue (type: boolean), ws_bill_customer_sk0 (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col1 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col1 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: boolean)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: catalog_sales
+                  properties:
+                    hive.sql.query SELECT TRUE AS "literalTrue", "t0"."cs_ship_customer_sk" AS "cs_ship_customer_sk0"
+FROM (SELECT "cs_sold_date_sk", "cs_sold_time_sk", "cs_ship_date_sk", "cs_bill_customer_sk", "cs_bill_cdemo_sk", "cs_bill_hdemo_sk", "cs_bill_addr_sk", "cs_ship_customer_sk", "cs_ship_cdemo_sk", "cs_ship_hdemo_sk", "cs_ship_addr_sk", "cs_call_center_sk", "cs_catalog_page_sk", "cs_ship_mode_sk", "cs_warehouse_sk", "cs_item_sk", "cs_promo_sk", "cs_order_number", "cs_quantity", "cs_wholesale_cost", "cs_list_price", "cs_sales_price", "cs_ext_discount_amt", "cs_ext_sales_price", "cs_ext_whole [...]
+FROM "CATALOG_SALES"
+WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
+                    hive.sql.query.fieldNames literalTrue,cs_ship_customer_sk0
+                    hive.sql.query.fieldTypes boolean,int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: cs_ship_customer_sk0 (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        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
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36, _col40
+                Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: _col40 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: int), _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                    outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Anti Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col32, _col33, _col34, _col35, _col36
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Top N Key Operator
+                  sort order: +++++
+                  keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                  null sort order: zzzzz
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  top n: 100
+                  Group By Operator
+                    aggregations: count()
+                    keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                    minReductionHashAggr: 0.99
+                    mode: hash
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                      null sort order: zzzzz
+                      sort order: +++++
+                      Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                      Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col5 (type: bigint)
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: char(1)), KEY._col1 (type: char(1)), KEY._col2 (type: char(20)), KEY._col3 (type: int), KEY._col4 (type: char(10))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col5 (type: bigint), _col3 (type: int), _col4 (type: char(10))
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col4 (type: int), _col6 (type: char(10))
+                    null sort order: zzzzz
+                    sort order: +++++
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col3 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: char(1)), KEY.reducesinkkey1 (type: char(1)), KEY.reducesinkkey2 (type: char(20)), VALUE._col0 (type: bigint), KEY.reducesinkkey3 (type: int), VALUE._col0 (type: bigint), KEY.reducesinkkey4 (type: char(10)), VALUE._col0 (type: bigint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cd_gender,
+       cd_marital_status,
+       cd_education_status,
+       Count(*) cnt1,
+       cd_purchase_estimate,
+       Count(*) cnt2,
+       cd_credit_rating,
+       Count(*) cnt3
+FROM   customer c,
+       customer_address ca,
+       customer_demographics
+WHERE  c.c_current_addr_sk = ca.ca_address_sk
+       AND ca_state IN ( 'CO', 'IL', 'MN' )
+       AND cd_demo_sk = c.c_current_cdemo_sk
+       AND EXISTS (SELECT *
+                   FROM   store_sales,
+                          date_dim
+                   WHERE  c.c_customer_sk = ss_customer_sk
+                          AND ss_sold_date_sk = d_date_sk
+                          AND d_year = 1999
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2)
+       AND ( NOT EXISTS (SELECT *
+                         FROM   web_sales,
+                                date_dim
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk
+                                AND ws_sold_date_sk = d_date_sk
+                                AND d_year = 1999
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2)
+             AND NOT EXISTS (SELECT *
+                             FROM   catalog_sales,
+                                    date_dim
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk
+                                    AND cs_sold_date_sk = d_date_sk
+                                    AND d_year = 1999
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+ORDER  BY cd_gender,
+          cd_marital_status,
+          cd_education_status,
+          cd_purchase_estimate,
+          cd_credit_rating
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
 PREHOOK: query: DROP TABLE catalog_sales
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@catalog_sales
diff --git a/ql/src/test/results/clientpositive/llap/lineage3.q.out b/ql/src/test/results/clientpositive/llap/lineage3.q.out
index 964075f..83233df 100644
--- a/ql/src/test/results/clientpositive/llap/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/llap/lineage3.q.out
@@ -202,7 +202,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-{"version":"1.0","engine":"tez","database":"default","hash":"723e79692e1de404c4ffb702097586da","queryText":"select * from src1 a\nwhere not 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],"expressio [...]
+{"version":"1.0","engine":"tez","database":"default","hash":"723e79692e1de404c4ffb702097586da","queryText":"select * from src1 a\nwhere not 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],"expressio [...]
 369	
 401	val_401
 406	val_406
@@ -401,3 +401,16 @@ Result schema has 2 fields, but we don't get as many dependencies
 Result schema has 2 fields, but we don't get as many dependencies
 Result schema has 2 fields, but we don't get as many dependencies
 {"version":"1.0","engine":"tez","database":"default","hash":"e540a88155ffa4bf6842a4fdf3bfe639","queryText":"from src_dp, src_dp1\ninsert into dest_dp1 partition (year) select first, word, year\ninsert into dest_dp2 partition (y, m) select first, word, year, month\ninsert into dest_dp3 partition (y=2, m, d) select first, word, month m, day d where year=2\ninsert into dest_dp2 partition (y=1, m) select f, w, m\ninsert into dest_dp1 partition (year=0) select f, w","edges":[{"sources":[11]," [...]
+PREHOOK: query: select * from src1 a
+where not exists
+  (select cint from alltypesorc b
+   where a.key = b.ctinyint + 300)
+and key > 300
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+{"version":"1.0","engine":"tez","database":"default","hash":"723e79692e1de404c4ffb702097586da","queryText":"select * from src1 a\nwhere not 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],"expressio [...]
+369	
+401	val_401
+406	val_406
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 68c9924..4164d1b 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
@@ -1643,6 +1643,374 @@ 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[127][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[128][bigTable=?] in task 'Map 4' is a cross product
+Warning: Map Join MAPJOIN[129][bigTable=?] in task 'Reducer 5' is a cross product
+PREHOOK: query: explain
+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)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null_n4
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+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)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null_n4
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Map 3 (BROADCAST_EDGE)
+        Map 4 <- Map 3 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (BROADCAST_EDGE), Reducer 8 (BROADCAST_EDGE)
+        Reducer 5 <- Map 1 (BROADCAST_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+        Reducer 7 <- Reducer 5 (SIMPLE_EDGE)
+        Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  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
+                    Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Anti Join 0 to 1
+                      keys:
+                        0 
+                        1 
+                      outputColumnNames: _col0, _col1, _col2
+                      input vertices:
+                        1 Map 3
+                      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
+                        Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                        Group By Operator
+                          aggregations: count()
+                          keys: _col1 (type: string), _col0 (type: string)
+                          minReductionHashAggr: 0.99
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                          Reduce Output Operator
+                            key expressions: _col0 (type: string), _col1 (type: string)
+                            null sort order: zz
+                            sort order: ++
+                            Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                            Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                            value expressions: _col2 (type: bigint)
+                  Filter Operator
+                    predicate: (key > '9') (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
+                      Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: string), _col1 (type: string)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: src_null_n4
+                  filterExpr: value is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: value is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: value (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                      Group By Operator
+                        keys: _col0 (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
+                          null sort order: 
+                          sort order: 
+                          Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: string)
+                        Reduce Output Operator
+                          null sort order: 
+                          sort order: 
+                          Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            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
+                  Filter Operator
+                    predicate: 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
+                      Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Anti Join 0 to 1
+                        keys:
+                          0 
+                          1 
+                        outputColumnNames: _col0, _col1, _col2
+                        input vertices:
+                          1 Map 3
+                        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
+                          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
+                            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)
+                              null sort order: zz
+                              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: all inputs
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col1 (type: string), _col0 (type: string), _col2 (type: bigint)
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                  Map Join Operator
+                    condition map:
+                         Left Outer Join 0 to 1
+                    keys:
+                      0 _col1 (type: string)
+                      1 _col0 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col4, _col5
+                    input vertices:
+                      1 Reducer 6
+                    Statistics: Num rows: 1 Data size: 608 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Left Outer Join 0 to 1
+                      keys:
+                        0 _col1 (type: string), _col2 (type: bigint)
+                        1 _col2 (type: string), _col0 (type: bigint)
+                      outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col7
+                      input vertices:
+                        1 Reducer 8
+                      Statistics: Num rows: 1 Data size: 668 Basic stats: COMPLETE Column stats: NONE
+                      Filter Operator
+                        predicate: (_col4 is null or (_col4 = 0L) or (_col7 is not null or _col2 is null or (_col5 < _col4)) is not true) (type: boolean)
+                        Statistics: Num rows: 1 Data size: 668 Basic stats: COMPLETE Column stats: NONE
+                        Select Operator
+                          expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint)
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 668 Basic stats: COMPLETE Column stats: NONE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 1 Data size: 668 Basic stats: COMPLETE Column stats: NONE
+                            table:
+                                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: string), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string)
+                  outputColumnNames: _col1
+                  Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    keys: _col1 (type: string)
+                    mode: complete
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      keys:
+                        0 
+                        1 
+                      outputColumnNames: _col0, _col1, _col2
+                      input vertices:
+                        0 Map 1
+                      residual filter predicates: {(_col1 <> _col2)}
+                      Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                      Select Operator
+                        expressions: _col0 (type: string), _col2 (type: string)
+                        outputColumnNames: _col0, _col2
+                        Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                        Group By Operator
+                          aggregations: count()
+                          keys: _col2 (type: string), _col0 (type: string)
+                          minReductionHashAggr: 0.99
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                          Reduce Output Operator
+                            key expressions: _col0 (type: string), _col1 (type: string)
+                            null sort order: zz
+                            sort order: ++
+                            Map-reduce partition columns: _col0 (type: string)
+                            Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                            value expressions: _col2 (type: bigint)
+                          Reduce Output Operator
+                            key expressions: _col0 (type: string), _col1 (type: string)
+                            null sort order: zz
+                            sort order: ++
+                            Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                            Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                            value expressions: _col2 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string), _col2 (type: bigint)
+                  outputColumnNames: _col1, _col2
+                  Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    aggregations: count(), count(_col2)
+                    keys: _col1 (type: string)
+                    mode: complete
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col1 (type: bigint), _col2 (type: bigint)
+        Reducer 7 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: string), _col2 (type: bigint)
+                  outputColumnNames: _col0, _col2
+                  Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: _col2 is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: _col0 (type: string), _col2 (type: bigint)
+                      outputColumnNames: _col1, _col2
+                      Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                      Group By Operator
+                        keys: _col1 (type: string), _col2 (type: bigint)
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
+                          Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+        Reducer 8 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: string), KEY._col1 (type: bigint)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: string)
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col2 (type: string), _col0 (type: bigint)
+                    null sort order: zz
+                    sort order: ++
+                    Map-reduce partition columns: _col2 (type: string), _col0 (type: bigint)
+                    Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: boolean)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+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)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null_n4
+#### A masked pattern was here ####
+POSTHOOK: 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)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null_n4
+#### A masked pattern was here ####
+5444	NULL	1
 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
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 95539e5..be03281 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
@@ -4457,6 +4457,1085 @@ STAGE PLANS:
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: ws1
+                  filterExpr: p_type is not null (type: boolean)
+                  Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: p_type is not null (type: boolean)
+                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                      Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col4 (type: string)
+                        Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: ws2
+                  filterExpr: (p_type is not null and p_retailprice is not null) (type: boolean)
+                  Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (p_type is not null and p_retailprice is not null) (type: boolean)
+                    Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: p_type (type: string), p_retailprice (type: double)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: string)
+                          Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col1 (type: double)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: wr1
+                  filterExpr: p_name is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: p_name is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: p_name (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                      Group By Operator
+                        keys: _col0 (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)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: string)
+                          Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col4 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10
+                residual filter predicates: {(_col7 <> _col10)}
+                Statistics: Num rows: 14 Data size: 8778 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                  Statistics: Num rows: 14 Data size: 8666 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col4 (type: string)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col4 (type: string)
+                    Statistics: Num rows: 14 Data size: 8666 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Anti Join 0 to 1
+                keys:
+                  0 _col4 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                Statistics: Num rows: 15 Data size: 9532 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 15 Data size: 9532 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select * from part ws1 where
+    exists (select * from part ws2 where ws1.p_type= ws2.p_type
+                            and ws1.p_retailprice <> ws2.p_retailprice)
+    and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Input: default@part_null
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part ws1 where
+    exists (select * from part ws2 where ws1.p_type= ws2.p_type
+                            and ws1.p_retailprice <> ws2.p_retailprice)
+    and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Input: default@part_null
+#### A masked pattern was here ####
+192697	almond antique blue firebrick mint	Manufacturer#5	Brand#52	MEDIUM BURNISHED TIN	31	LG DRUM	1789.69	ickly ir
+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
+PREHOOK: query: explain select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_null
+PREHOOK: Input: default@tempty
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_null
+POSTHOOK: Input: default@tempty
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
+        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: part_null
+                  filterExpr: p_size is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: p_size is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                      Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col5 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col5 (type: int)
+                        Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+                    Group By Operator
+                      keys: p_size (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        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: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: tempty
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    Group By Operator
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col5 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10
+                Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+                Filter Operator
+                  predicate: _col10 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                    Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  value expressions: _col0 (type: boolean)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+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_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_null
+PREHOOK: Input: default@tempty
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_null
+POSTHOOK: Input: default@tempty
+#### A masked pattern was here ####
+105685	almond antique violet chocolate turquoise	Manufacturer#2	Brand#22	MEDIUM ANODIZED COPPER	14	MED CAN	1690.68	ly pending requ
+110592	almond antique salmon chartreuse burlywood	Manufacturer#1	Brand#15	PROMO BURNISHED NICKEL	6	JUMBO PKG	1602.59	 to the furiously
+112398	almond antique metallic orange dim	Manufacturer#3	Brand#32	MEDIUM BURNISHED BRASS	19	JUMBO JAR	1410.39	ole car
+121152	almond antique burnished rose metallic	Manufacturer#1	Brand#14	PROMO PLATED TIN	2	JUMBO BOX	1173.15	e pinto beans h
+121152	almond antique burnished rose metallic	Manufacturer#1	Brand#14	PROMO PLATED TIN	2	JUMBO BOX	1173.15	e pinto beans h
+132666	almond aquamarine rose maroon antique	Manufacturer#2	Brand#24	SMALL POLISHED NICKEL	25	MED BOX	1698.66	even 
+144293	almond antique olive coral navajo	Manufacturer#3	Brand#34	STANDARD POLISHED STEEL	45	JUMBO CAN	1337.29	ag furiously about 
+146985	almond aquamarine midnight light salmon	Manufacturer#2	Brand#23	MEDIUM BURNISHED COPPER	2	SM CASE	2031.98	s cajole caref
+15103	almond aquamarine dodger light gainsboro	Manufacturer#5	Brand#53	ECONOMY BURNISHED STEEL	46	LG PACK	1018.1	packages hinder carefu
+155733	almond antique sky peru orange	Manufacturer#5	Brand#53	SMALL PLATED BRASS	2	WRAP DRUM	1788.73	furiously. bra
+17273	almond antique forest lavender goldenrod	Manufacturer#3	Brand#35	PROMO ANODIZED TIN	14	JUMBO CASE	1190.27	along the
+17927	almond aquamarine yellow dodger mint	Manufacturer#4	Brand#41	ECONOMY BRUSHED COPPER	7	SM PKG	1844.92	ites. eve
+191709	almond antique violet turquoise frosted	Manufacturer#2	Brand#22	ECONOMY POLISHED STEEL	40	MED BOX	1800.7	 haggle
+192697	almond antique blue firebrick mint	Manufacturer#5	Brand#52	MEDIUM BURNISHED TIN	31	LG DRUM	1789.69	ickly ir
+195606	almond aquamarine sandy cyan gainsboro	Manufacturer#2	Brand#25	STANDARD PLATED TIN	18	SM PKG	1701.6	ic de
+33357	almond azure aquamarine papaya violet	Manufacturer#4	Brand#41	STANDARD ANODIZED TIN	12	WRAP CASE	1290.35	reful
+40982	almond antique misty red olive	Manufacturer#3	Brand#32	ECONOMY PLATED COPPER	1	LG PKG	1922.98	c foxes can s
+42669	almond antique medium spring khaki	Manufacturer#5	Brand#51	STANDARD BURNISHED TIN	6	MED CAN	1611.66	sits haggl
+45261	almond aquamarine floral ivory bisque	Manufacturer#4	Brand#42	SMALL PLATED STEEL	27	WRAP CASE	1206.26	careful
+48427	almond antique violet mint lemon	Manufacturer#4	Brand#42	PROMO POLISHED STEEL	39	SM CASE	1375.42	hely ironic i
+49671	almond antique gainsboro frosted violet	Manufacturer#4	Brand#41	SMALL BRUSHED BRASS	10	SM BOX	1620.67	ccounts run quick
+65667	almond aquamarine pink moccasin thistle	Manufacturer#1	Brand#12	LARGE BURNISHED STEEL	42	JUMBO CASE	1632.66	e across the expr
+78486	almond azure blanched chiffon midnight	Manufacturer#5	Brand#52	LARGE BRUSHED BRASS	23	MED BAG	1464.48	hely blith
+78487	NULL	Manufacturer#6	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
+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[41][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 IN (select p_name from part_null) AND NOT EXISTS (select c from tempty)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_null
+PREHOOK: Input: default@tempty
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_null
+POSTHOOK: Input: default@tempty
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE)
+        Reducer 6 <- Map 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: part_null
+                  filterExpr: p_name is not null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: p_name is not null (type: boolean)
+                    Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                      Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (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)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: tempty
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    Group By Operator
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col1 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10
+                Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+                Filter Operator
+                  predicate: _col10 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                    Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  value expressions: _col0 (type: boolean)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+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_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_null
+PREHOOK: Input: default@tempty
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_null
+POSTHOOK: Input: default@tempty
+#### A masked pattern was here ####
+105685	almond antique violet chocolate turquoise	Manufacturer#2	Brand#22	MEDIUM ANODIZED COPPER	14	MED CAN	1690.68	ly pending requ
+110592	almond antique salmon chartreuse burlywood	Manufacturer#1	Brand#15	PROMO BURNISHED NICKEL	6	JUMBO PKG	1602.59	 to the furiously
+112398	almond antique metallic orange dim	Manufacturer#3	Brand#32	MEDIUM BURNISHED BRASS	19	JUMBO JAR	1410.39	ole car
+121152	almond antique burnished rose metallic	Manufacturer#1	Brand#14	PROMO PLATED TIN	2	JUMBO BOX	1173.15	e pinto beans h
+121152	almond antique burnished rose metallic	Manufacturer#1	Brand#14	PROMO PLATED TIN	2	JUMBO BOX	1173.15	e pinto beans h
+132666	almond aquamarine rose maroon antique	Manufacturer#2	Brand#24	SMALL POLISHED NICKEL	25	MED BOX	1698.66	even 
+144293	almond antique olive coral navajo	Manufacturer#3	Brand#34	STANDARD POLISHED STEEL	45	JUMBO CAN	1337.29	ag furiously about 
+146985	almond aquamarine midnight light salmon	Manufacturer#2	Brand#23	MEDIUM BURNISHED COPPER	2	SM CASE	2031.98	s cajole caref
+15103	almond aquamarine dodger light gainsboro	Manufacturer#5	Brand#53	ECONOMY BURNISHED STEEL	46	LG PACK	1018.1	packages hinder carefu
+155733	almond antique sky peru orange	Manufacturer#5	Brand#53	SMALL PLATED BRASS	2	WRAP DRUM	1788.73	furiously. bra
+17273	almond antique forest lavender goldenrod	Manufacturer#3	Brand#35	PROMO ANODIZED TIN	14	JUMBO CASE	1190.27	along the
+17927	almond aquamarine yellow dodger mint	Manufacturer#4	Brand#41	ECONOMY BRUSHED COPPER	7	SM PKG	1844.92	ites. eve
+191709	almond antique violet turquoise frosted	Manufacturer#2	Brand#22	ECONOMY POLISHED STEEL	40	MED BOX	1800.7	 haggle
+192697	almond antique blue firebrick mint	Manufacturer#5	Brand#52	MEDIUM BURNISHED TIN	31	LG DRUM	1789.69	ickly ir
+195606	almond aquamarine sandy cyan gainsboro	Manufacturer#2	Brand#25	STANDARD PLATED TIN	18	SM PKG	1701.6	ic de
+33357	almond azure aquamarine papaya violet	Manufacturer#4	Brand#41	STANDARD ANODIZED TIN	12	WRAP CASE	1290.35	reful
+40982	almond antique misty red olive	Manufacturer#3	Brand#32	ECONOMY PLATED COPPER	1	LG PKG	1922.98	c foxes can s
+42669	almond antique medium spring khaki	Manufacturer#5	Brand#51	STANDARD BURNISHED TIN	6	MED CAN	1611.66	sits haggl
+45261	almond aquamarine floral ivory bisque	Manufacturer#4	Brand#42	SMALL PLATED STEEL	27	WRAP CASE	1206.26	careful
+48427	almond antique violet mint lemon	Manufacturer#4	Brand#42	PROMO POLISHED STEEL	39	SM CASE	1375.42	hely ironic i
+49671	almond antique gainsboro frosted violet	Manufacturer#4	Brand#41	SMALL BRUSHED BRASS	10	SM BOX	1620.67	ccounts run quick
+65667	almond aquamarine pink moccasin thistle	Manufacturer#1	Brand#12	LARGE BURNISHED STEEL	42	JUMBO CASE	1632.66	e across the expr
+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
+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
+PREHOOK: query: explain
+select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 8 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+        Reducer 7 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  filterExpr: ((value is not null and key is not null) or (key > '9') or value is not null) (type: boolean)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (value is not null and key is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string), _col1 (type: string)
+                        null sort order: zz
+                        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
+                  Filter Operator
+                    predicate: (key > '9') (type: boolean)
+                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _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.5
+                      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)
+                        null sort order: z
+                        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: all inputs
+        Map 8 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  filterExpr: (value is not null and key is not null) (type: boolean)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (value is not null and key is not null) (type: boolean)
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: string)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string), _col1 (type: string)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                          Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col0 (type: string), _col1 (type: string)
+                  1 _col0 (type: string), _col1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  keys: _col0 (type: string), _col1 (type: string)
+                  minReductionHashAggr: 0.0
+                  mode: hash
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: string), _col1 (type: string)
+                    null sort order: zz
+                    sort order: ++
+                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                    Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col2 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string), KEY._col1 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: _col2 is not null (type: boolean)
+                  Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col2 (type: bigint)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col2 (type: bigint)
+                    Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: string), _col1 (type: string)
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col2 (type: bigint)
+                  1 _col0 (type: bigint)
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 21 Data size: 3906 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 21 Data size: 3906 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col1 (type: string)
+                  1 _col1 (type: string)
+                outputColumnNames: _col0, _col2
+                Statistics: Num rows: 250 Data size: 22418 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: _col2 is null (type: boolean)
+                  Statistics: Num rows: 84 Data size: 7536 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 84 Data size: 7536 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count()
+                      keys: _col0 (type: string)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 42 Data size: 3990 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 42 Data size: 3990 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 42 Data size: 3990 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: bigint)
+                  outputColumnNames: _col1
+                  Statistics: Num rows: 42 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: _col1 is not null (type: boolean)
+                    Statistics: Num rows: 42 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: _col1 (type: bigint)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 42 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: bigint)
+                        minReductionHashAggr: 0.5
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: bigint)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: bigint)
+                          Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 7 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: true (type: boolean), _col0 (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: string)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: string)
+                    Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: boolean)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[49][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: explain select count(*)  from src
+    where src.key in (select key from src s1 where s1.key > '9')
+        or src.value is not null
+        or not exists(select key from src)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain select count(*)  from src
+    where src.key in (select key from src s1 where s1.key > '9')
+        or src.value is not null
+        or not exists(select key from src)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Reducer 6 (XPROD_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 6 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+        Reducer 7 <- Map 1 (SIMPLE_EDGE)
+        Reducer 8 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: string), _col1 (type: string)
+                  Filter Operator
+                    predicate: (key > '9') (type: boolean)
+                    Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: count()
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          null sort order: 
+                          sort order: 
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: bigint)
+                  Filter Operator
+                    predicate: (key > '9') (type: boolean)
+                    Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: key (type: string)
+                      minReductionHashAggr: 0.5
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: true (type: boolean)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: boolean)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: boolean)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                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)
+                  null sort order: z
+                  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: bigint)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col1, _col2, _col4
+                Statistics: Num rows: 631 Data size: 62997 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 631 Data size: 62997 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string), _col2 (type: bigint), _col4 (type: boolean)
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col1, _col2, _col4, _col5
+                Statistics: Num rows: 631 Data size: 65521 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: (_col5 is null or ((_col2 <> 0L) and _col4 is not null) or _col1 is not null) (type: boolean)
+                  Statistics: Num rows: 631 Data size: 65521 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    Statistics: Num rows: 631 Data size: 65521 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count()
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: bigint)
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: bigint)
+        Reducer 7 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: string)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: string), true (type: boolean)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 83 Data size: 7553 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: string)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: string)
+                    Statistics: Num rows: 83 Data size: 7553 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: boolean)
+        Reducer 8 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: boolean)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  null sort order: 
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: boolean)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[49][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: select count(*)  from src
+    where src.key in (select key from src s1 where s1.key > '9')
+        or src.value is not null
+        or not exists(select key from src)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*)  from src
+    where src.key in (select key from src s1 where s1.key > '9')
+        or src.value is not null
+        or not exists(select key from src)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+500
+PREHOOK: query: explain select * from part ws1 where
+    exists (select * from part ws2 where ws1.p_type= ws2.p_type
+                            and ws1.p_retailprice <> ws2.p_retailprice)
+    and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Input: default@part_null
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from part ws1 where
+    exists (select * from part ws2 where ws1.p_type= ws2.p_type
+                            and ws1.p_retailprice <> ws2.p_retailprice)
+    and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Input: default@part_null
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 6 <- Map 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out b/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out
index 226c00b..faea4f4 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out
@@ -29,8 +29,7 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -48,21 +47,33 @@ STAGE PLANS:
                       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: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: ((value > 'val_2') and key is not null) (type: boolean)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((value > 'val_2') and key is not null) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: string), value (type: string)
-                      minReductionHashAggr: 0.0
-                      mode: hash
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string), _col1 (type: string)
-                        null sort order: zz
-                        sort order: ++
-                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: string)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0, _col1
                         Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string), _col1 (type: string)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                          Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -70,45 +81,19 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Anti Join 0 to 1
                 keys:
                   0 _col0 (type: string), _col1 (type: string)
-                  1 _col1 (type: string), _col2 (type: string)
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 631 Data size: 112846 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: _col2 is null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89420 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    File Output Operator
-                      compressed: false
-                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                      table:
-                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: string), KEY._col1 (type: string)
-                mode: mergepartial
+                  1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: true (type: boolean), _col0 (type: string), _col1 (type: string)
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string), _col2 (type: string)
-                    null sort order: zz
-                    sort order: ++
-                    Map-reduce partition columns: _col1 (type: string), _col2 (type: string)
-                    Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: boolean)
+                Statistics: Num rows: 369 Data size: 65682 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 369 Data size: 65682 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -319,7 +304,7 @@ STAGE PLANS:
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: string)
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -328,26 +313,19 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Anti Join 0 to 1
                 keys:
                   0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 566 Data size: 101016 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: _col2 is null (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89236 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                    File Output Operator
-                      compressed: false
-                      Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                      table:
-                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 434 Data size: 77252 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 434 Data size: 77252 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -358,24 +336,20 @@ STAGE PLANS:
                 Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string)
-                  outputColumnNames: _col1
-                  Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 83 Data size: 7553 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    keys: _col1 (type: string)
-                    mode: complete
+                    keys: _col0 (type: string)
+                    minReductionHashAggr: 0.5060241
+                    mode: hash
                     outputColumnNames: _col0
                     Statistics: Num rows: 41 Data size: 3731 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: true (type: boolean), _col0 (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
-                        Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: boolean)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 41 Data size: 3731 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -522,7 +496,7 @@ POSTHOOK: Input: default@src
 199	val_199
 199	val_199
 2	val_2
-Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: explain
 select *
 from src b
@@ -554,8 +528,7 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 3 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (XPROD_EDGE), Map 3 (XPROD_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -572,21 +545,32 @@ STAGE PLANS:
                       sort order: 
                       Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: string), _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  filterExpr: ((value > 'val_2') and key is not null) (type: boolean)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((value > 'val_2') and key is not null) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: string), value (type: string)
-                      minReductionHashAggr: 0.0
-                      mode: hash
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string), _col1 (type: string)
-                        null sort order: zz
-                        sort order: ++
-                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: string)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0, _col1
                         Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          null sort order: 
+                          sort order: 
+                          Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: string), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -594,44 +578,24 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Anti Join 0 to 1
                 keys:
                   0 
                   1 
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                residual filter predicates: {(_col3 > _col0)} {(_col1 <> _col4)}
-                Statistics: Num rows: 41500 Data size: 14940000 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: _col2 is null (type: boolean)
-                  Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
-                    File Output Operator
-                      compressed: false
-                      Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
-                      table:
-                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: string), KEY._col1 (type: string)
-                mode: mergepartial
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col1, _col2, _col3
+                residual filter predicates: {(_col2 > _col0)} {(_col1 <> _col3)}
+                Statistics: Num rows: 13833 Data size: 4924548 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: true (type: boolean), _col0 (type: string), _col1 (type: string)
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    null sort order: 
-                    sort order: 
-                    Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: boolean), _col1 (type: string), _col2 (type: string)
+                  expressions: _col0 (type: string), _col1 (type: string)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 13833 Data size: 2462274 Basic stats: COMPLETE Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 13833 Data size: 2462274 Basic stats: COMPLETE Column stats: COMPLETE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -639,7 +603,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
 PREHOOK: query: select *
 from src b
 where not exists
@@ -662,8 +626,8 @@ POSTHOOK: Input: default@src
 #### A masked pattern was here ####
 98	val_98
 98	val_98
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: explain SELECT p1.p_name FROM part p1 LEFT JOIN (select p_type as p_col from part ) p2 WHERE NOT EXISTS
                 (select pp1.p_type as p_col from part pp1 where pp1.p_partkey = p2.p_col)
 PREHOOK: type: QUERY
@@ -683,12 +647,11 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 8 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-        Reducer 6 <- Map 1 (XPROD_EDGE), Map 8 (XPROD_EDGE)
-        Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 7 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (XPROD_EDGE), Map 7 (XPROD_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -726,7 +689,7 @@ STAGE PLANS:
                       Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 8 
+        Map 7 
             Map Operator Tree:
                 TableScan
                   alias: part
@@ -777,26 +740,19 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Anti Join 0 to 1
                 keys:
                   0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 1014 Data size: 124050 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: _col2 is null (type: boolean)
-                  Statistics: Num rows: 676 Data size: 82700 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 676 Data size: 81796 Basic stats: COMPLETE Column stats: COMPLETE
-                    File Output Operator
-                      compressed: false
-                      Statistics: Num rows: 676 Data size: 81796 Basic stats: COMPLETE Column stats: COMPLETE
-                      table:
-                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  1 _col0 (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 338 Data size: 40898 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 338 Data size: 40898 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 4 
             Execution mode: llap
             Reduce Operator Tree:
@@ -808,38 +764,23 @@ STAGE PLANS:
                   1 _col1 (type: double)
                 outputColumnNames: _col1
                 Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  keys: _col1 (type: string)
-                  minReductionHashAggr: 0.5
-                  mode: hash
+                Select Operator
+                  expressions: _col1 (type: string)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 12 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: string)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    keys: _col0 (type: string)
+                    minReductionHashAggr: 0.5
+                    mode: hash
+                    outputColumnNames: _col0
                     Statistics: Num rows: 12 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 12 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 5 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: string)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 12 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: true (type: boolean), _col0 (type: string)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 12 Data size: 1296 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: string)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: string)
-                    Statistics: Num rows: 12 Data size: 1296 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: boolean)
-        Reducer 6 
             Execution mode: llap
             Reduce Operator Tree:
               Merge Join Operator
@@ -862,7 +803,7 @@ STAGE PLANS:
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 7 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -888,8 +829,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
 PREHOOK: query: SELECT p1.p_name FROM part p1 LEFT JOIN (select p_type as p_col from part ) p2 WHERE NOT EXISTS
                 (select pp1.p_type as p_col from part pp1 where pp1.p_partkey = p2.p_col)
 PREHOOK: type: QUERY
@@ -1608,14 +1549,12 @@ POSTHOOK: Input: default@tschema
 CBO PLAN:
 HiveProject(eid=[_UTF-16LE'empno':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], id=[$0])
   HiveAggregate(group=[{0}])
-    HiveFilter(condition=[IS NULL($1)])
-      HiveJoin(condition=[=($0, $2)], joinType=[left], algorithm=[none], cost=[not available])
-        HiveProject(id=[$0])
-          HiveTableScan(table=[[default, tschema]], table:alias=[a])
-        HiveProject(literalTrue=[true], id0=[$0])
-          HiveAggregate(group=[{0}])
-            HiveFilter(condition=[IS NOT NULL($0)])
-              HiveTableScan(table=[[default, tschema]], table:alias=[c])
+    HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+      HiveProject(id=[$0])
+        HiveTableScan(table=[[default, tschema]], table:alias=[a])
+      HiveProject(literalTrue=[true], id0=[$0])
+        HiveFilter(condition=[IS NOT NULL($0)])
+          HiveTableScan(table=[[default, tschema]], table:alias=[c])
 
 PREHOOK: query: select distinct 'empno' as eid, a.id from tschema a
     where NOT EXISTS (select c.id from tschema c where a.id=c.id)
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notexists_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_notexists_having.q.out
index f24ad46..4a2d993 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notexists_having.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notexists_having.q.out
@@ -31,7 +31,8 @@ STAGE PLANS:
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -55,72 +56,59 @@ STAGE PLANS:
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-        Map 3 
-            Map Operator Tree:
-                TableScan
-                  alias: a
-                  filterExpr: ((value > 'val_12') and key is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((value > 'val_12') and key is not null) (type: boolean)
                     Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: string), value (type: string)
-                      minReductionHashAggr: 0.0
-                      mode: hash
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: string), _col1 (type: string)
-                        null sort order: zz
-                        sort order: ++
-                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: string)
+                        minReductionHashAggr: 0.0
+                        mode: hash
+                        outputColumnNames: _col0, _col1
                         Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string), _col1 (type: string)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                          Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: true (type: boolean), _col0 (type: string), _col1 (type: string)
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
-                  Dummy Store
+                Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: zz
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+                  Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
+        Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: string), KEY._col1 (type: string)
-                mode: mergepartial
+              Merge Join Operator
+                condition map:
+                     Anti Join 0 to 1
+                keys:
+                  0 _col0 (type: string), _col1 (type: string)
+                  1 _col0 (type: string), _col1 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
-                Merge Join Operator
-                  condition map:
-                       Left Outer Join 0 to 1
-                  keys:
-                    0 _col0 (type: string), _col1 (type: string)
-                    1 _col1 (type: string), _col2 (type: string)
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 333 Data size: 59610 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: _col2 is null (type: boolean)
-                    Statistics: Num rows: 250 Data size: 44752 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: _col0 (type: string), _col1 (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
-                      File Output Operator
-                        compressed: false
-                        Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
-                        table:
-                            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                Statistics: Num rows: 167 Data size: 29726 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 167 Data size: 29726 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0
     Fetch Operator
@@ -236,7 +224,7 @@ STAGE PLANS:
                         key expressions: _col0 (type: string), _col1 (type: string)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: string)
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -264,26 +252,19 @@ STAGE PLANS:
             Reduce Operator Tree:
               Merge Join Operator
                 condition map:
-                     Left Outer Join 0 to 1
+                     Anti Join 0 to 1
                 keys:
                   0 _col1 (type: string)
-                  1 _col1 (type: string)
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 291 Data size: 51966 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: _col2 is null (type: boolean)
-                  Statistics: Num rows: 250 Data size: 44644 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: string), _col1 (type: string)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
-                    File Output Operator
-                      compressed: false
-                      Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
-                      table:
-                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 209 Data size: 37202 Basic stats: COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 209 Data size: 37202 Basic stats: COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -294,24 +275,20 @@ STAGE PLANS:
                 Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string)
-                  outputColumnNames: _col1
-                  Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 83 Data size: 7553 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    keys: _col1 (type: string)
-                    mode: complete
+                    keys: _col0 (type: string)
+                    minReductionHashAggr: 0.5060241
+                    mode: hash
                     outputColumnNames: _col0
                     Statistics: Num rows: 41 Data size: 3731 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: true (type: boolean), _col0 (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
-                        Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: boolean)
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 41 Data size: 3731 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
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 0f166ce..4ef2b56 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
@@ -4461,6 +4461,150 @@ POSTHOOK: Input: default@part
 POSTHOOK: Input: default@part_null_n0
 #### A masked pattern was here ####
 192697	almond antique blue firebrick mint	Manufacturer#5	Brand#52	MEDIUM BURNISHED TIN	31	LG DRUM	1789.69	ickly ir
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: explain select * from part where (select i from tnull_n0 limit 1) is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Input: default@tnull_n0
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from part where (select i from tnull_n0 limit 1) is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Input: default@tnull_n0
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: part
+                  Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                    Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: tnull_n0
+                  Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: i (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Limit
+                      Number of rows: 1
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        null sort order: 
+                        sort order: 
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                        TopN Hash Memory Usage: 0.1
+                        value expressions: _col0 (type: int)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+                Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
+                Filter Operator
+                  predicate: _col9 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 623 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+                    Statistics: Num rows: 1 Data size: 619 Basic stats: COMPLETE Column stats: COMPLETE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 619 Basic stats: COMPLETE Column stats: COMPLETE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                Limit
+                  Number of rows: 1
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    null sort order: 
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: int)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: select * from part where (select i from tnull_n0 limit 1) is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Input: default@tnull_n0
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part where (select i from tnull_n0 limit 1) is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Input: default@tnull_n0
+#### A masked pattern was here ####
+105685	almond antique violet chocolate turquoise	Manufacturer#2	Brand#22	MEDIUM ANODIZED COPPER	14	MED CAN	1690.68	ly pending requ
+110592	almond antique salmon chartreuse burlywood	Manufacturer#1	Brand#15	PROMO BURNISHED NICKEL	6	JUMBO PKG	1602.59	 to the furiously
+112398	almond antique metallic orange dim	Manufacturer#3	Brand#32	MEDIUM BURNISHED BRASS	19	JUMBO JAR	1410.39	ole car
+121152	almond antique burnished rose metallic	Manufacturer#1	Brand#14	PROMO PLATED TIN	2	JUMBO BOX	1173.15	e pinto beans h
+121152	almond antique burnished rose metallic	Manufacturer#1	Brand#14	PROMO PLATED TIN	2	JUMBO BOX	1173.15	e pinto beans h
+132666	almond aquamarine rose maroon antique	Manufacturer#2	Brand#24	SMALL POLISHED NICKEL	25	MED BOX	1698.66	even 
+144293	almond antique olive coral navajo	Manufacturer#3	Brand#34	STANDARD POLISHED STEEL	45	JUMBO CAN	1337.29	ag furiously about 
+146985	almond aquamarine midnight light salmon	Manufacturer#2	Brand#23	MEDIUM BURNISHED COPPER	2	SM CASE	2031.98	s cajole caref
+15103	almond aquamarine dodger light gainsboro	Manufacturer#5	Brand#53	ECONOMY BURNISHED STEEL	46	LG PACK	1018.1	packages hinder carefu
+155733	almond antique sky peru orange	Manufacturer#5	Brand#53	SMALL PLATED BRASS	2	WRAP DRUM	1788.73	furiously. bra
+17273	almond antique forest lavender goldenrod	Manufacturer#3	Brand#35	PROMO ANODIZED TIN	14	JUMBO CASE	1190.27	along the
+17927	almond aquamarine yellow dodger mint	Manufacturer#4	Brand#41	ECONOMY BRUSHED COPPER	7	SM PKG	1844.92	ites. eve
+191709	almond antique violet turquoise frosted	Manufacturer#2	Brand#22	ECONOMY POLISHED STEEL	40	MED BOX	1800.7	 haggle
+192697	almond antique blue firebrick mint	Manufacturer#5	Brand#52	MEDIUM BURNISHED TIN	31	LG DRUM	1789.69	ickly ir
+195606	almond aquamarine sandy cyan gainsboro	Manufacturer#2	Brand#25	STANDARD PLATED TIN	18	SM PKG	1701.6	ic de
+33357	almond azure aquamarine papaya violet	Manufacturer#4	Brand#41	STANDARD ANODIZED TIN	12	WRAP CASE	1290.35	reful
+40982	almond antique misty red olive	Manufacturer#3	Brand#32	ECONOMY PLATED COPPER	1	LG PKG	1922.98	c foxes can s
+42669	almond antique medium spring khaki	Manufacturer#5	Brand#51	STANDARD BURNISHED TIN	6	MED CAN	1611.66	sits haggl
+45261	almond aquamarine floral ivory bisque	Manufacturer#4	Brand#42	SMALL PLATED STEEL	27	WRAP CASE	1206.26	careful
+48427	almond antique violet mint lemon	Manufacturer#4	Brand#42	PROMO POLISHED STEEL	39	SM CASE	1375.42	hely ironic i
+49671	almond antique gainsboro frosted violet	Manufacturer#4	Brand#41	SMALL BRUSHED BRASS	10	SM BOX	1620.67	ccounts run quick
+65667	almond aquamarine pink moccasin thistle	Manufacturer#1	Brand#12	LARGE BURNISHED STEEL	42	JUMBO CASE	1632.66	e across the expr
+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
+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
 PREHOOK: query: drop table tnull_n0
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@tnull_n0
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
index 2bc16ff..5fb0f8e 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
@@ -72,30 +72,28 @@ POSTHOOK: Input: default@date_dim
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
-  HiveFilter(condition=[IS NULL($13)])
-    HiveJoin(condition=[=($4, $14)], joinType=[left], algorithm=[none], cost=[not available])
-      HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
-        HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
-          HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
-                HiveFilter(condition=[AND(=($8, _UTF-16LE'NY'), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-              HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
-                  HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($10), IS NOT NULL($17))])
-                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
-                HiveProject(d_date_sk=[$0], d_date=[$2])
-                  HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00:TIMESTAMP(9), 2001-05-31 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-            HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
-              HiveFilter(condition=[AND(IN($25, _UTF-16LE'Ziebach County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Levy County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Huron County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Franklin Parish':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Daviess County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
-        HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
-          HiveFilter(condition=[AND(IS NOT NULL($14), IS NOT NULL($17))])
-            HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
-      HiveProject(literalTrue=[true], cr_order_number0=[$0])
-        HiveAggregate(group=[{16}])
-          HiveFilter(condition=[IS NOT NULL($16)])
-            HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
+  HiveAntiJoin(condition=[=($4, $14)], joinType=[anti])
+    HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
+      HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
+        HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
+              HiveFilter(condition=[AND(=($8, _UTF-16LE'NY'), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+            HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
+                HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($10), IS NOT NULL($17))])
+                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
+              HiveProject(d_date_sk=[$0], d_date=[$2])
+                HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00:TIMESTAMP(9), 2001-05-31 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
+            HiveFilter(condition=[AND(IN($25, _UTF-16LE'Ziebach County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Levy County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Huron County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Franklin Parish':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Daviess County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), IS NOT NULL($0))])
+              HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
+      HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
+        HiveFilter(condition=[AND(IS NOT NULL($14), IS NOT NULL($17))])
+          HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
+    HiveProject(literalTrue=[true], cr_order_number0=[$16])
+      HiveFilter(condition=[IS NOT NULL($16)])
+        HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query69.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query69.q.out
index e114e34..e75ff38 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query69.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query69.q.out
@@ -110,47 +110,44 @@ CBO PLAN:
 HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$4], sort4=[$6], dir0=[ASC], dir1=[ASC], dir2=[ASC], dir3=[ASC], dir4=[ASC], fetch=[100])
   HiveProject(cd_gender=[$0], cd_marital_status=[$1], cd_education_status=[$2], cnt1=[$5], cd_purchase_estimate=[$3], cnt2=[$5], cd_credit_rating=[$4], cnt3=[$5])
     HiveAggregate(group=[{6, 7, 8, 9, 10}], agg#0=[count()])
-      HiveFilter(condition=[IS NULL($13)])
-        HiveJoin(condition=[=($0, $14)], joinType=[left], algorithm=[none], cost=[not available])
-          HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], cd_demo_sk=[$5], cd_gender=[$6], cd_marital_status=[$7], cd_education_status=[$8], cd_purchase_estimate=[$9], cd_credit_rating=[$10], literalTrue=[$11], ws_bill_customer_sk0=[$12])
-            HiveFilter(condition=[IS NULL($11)])
-              HiveJoin(condition=[=($0, $12)], joinType=[left], algorithm=[none], cost=[not available])
-                HiveSemiJoin(condition=[=($0, $11)], joinType=[semi])
-                  HiveJoin(condition=[=($5, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_addr_sk=[$4])
-                        HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($4), IS NOT NULL($0))])
-                          HiveTableScan(table=[[default, customer]], table:alias=[c])
-                      HiveProject(ca_address_sk=[$0], ca_state=[$8])
-                        HiveFilter(condition=[AND(IN($8, _UTF-16LE'CO':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'IL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'MN':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), IS NOT NULL($0))])
-                          HiveTableScan(table=[[default, customer_address]], table:alias=[ca])
-                    HiveProject(cd_demo_sk=[$0], cd_gender=[$1], cd_marital_status=[$2], cd_education_status=[$3], cd_purchase_estimate=[$4], cd_credit_rating=[$5])
-                      HiveFilter(condition=[IS NOT NULL($0)])
-                        HiveTableScan(table=[[default, customer_demographics]], table:alias=[customer_demographics])
-                  HiveProject(ss_customer_sk0=[$1])
-                    HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
-                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
-                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                      HiveProject(d_date_sk=[$0])
-                        HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                HiveProject(literalTrue=[true], ws_bill_customer_sk0=[$0])
-                  HiveAggregate(group=[{1}])
-                    HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                      HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
-                        HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
-                          HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                      HiveProject(d_date_sk=[$0])
-                        HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
-                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-          HiveProject(literalTrue=[true], cs_ship_customer_sk0=[$0])
-            HiveAggregate(group=[{1}])
-              HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(cs_sold_date_sk=[$0], cs_ship_customer_sk=[$7])
-                  HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                HiveProject(d_date_sk=[$0])
-                  HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+      HiveAntiJoin(condition=[=($0, $14)], joinType=[anti])
+        HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], cd_demo_sk=[$5], cd_gender=[$6], cd_marital_status=[$7], cd_education_status=[$8], cd_purchase_estimate=[$9], cd_credit_rating=[$10], literalTrue=[$11], ws_bill_customer_sk0=[$12])
+          HiveFilter(condition=[IS NULL($11)])
+            HiveJoin(condition=[=($0, $12)], joinType=[left], algorithm=[none], cost=[not available])
+              HiveSemiJoin(condition=[=($0, $11)], joinType=[semi])
+                HiveJoin(condition=[=($5, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_addr_sk=[$4])
+                      HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($4), IS NOT NULL($0))])
+                        HiveTableScan(table=[[default, customer]], table:alias=[c])
+                    HiveProject(ca_address_sk=[$0], ca_state=[$8])
+                      HiveFilter(condition=[AND(IN($8, _UTF-16LE'CO':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'IL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'MN':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), IS NOT NULL($0))])
+                        HiveTableScan(table=[[default, customer_address]], table:alias=[ca])
+                  HiveProject(cd_demo_sk=[$0], cd_gender=[$1], cd_marital_status=[$2], cd_education_status=[$3], cd_purchase_estimate=[$4], cd_credit_rating=[$5])
+                    HiveFilter(condition=[IS NOT NULL($0)])
+                      HiveTableScan(table=[[default, customer_demographics]], table:alias=[customer_demographics])
+                HiveProject(ss_customer_sk0=[$1])
+                  HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                    HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
+                      HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+                        HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                    HiveProject(d_date_sk=[$0])
+                      HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
+                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+              HiveProject(literalTrue=[true], ws_bill_customer_sk0=[$1])
+                HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                  HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
+                    HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
+                      HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                  HiveProject(d_date_sk=[$0])
+                    HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
+                      HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+        HiveProject(literalTrue=[true], cs_ship_customer_sk0=[$1])
+          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(cs_sold_date_sk=[$0], cs_ship_customer_sk=[$7])
+              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query78.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query78.q.out
index 180204c..df6fb01 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query78.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query78.q.out
@@ -142,14 +142,13 @@ HiveSortLimit(fetch=[100])
                     HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
                       HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                   HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_quantity=[$4], ss_wholesale_cost=[$5], ss_sales_price=[$6])
-                    HiveFilter(condition=[IS NULL($8)])
-                      HiveJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[left], algorithm=[none], cost=[not available])
-                        HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_ticket_number=[$9], ss_quantity=[$10], ss_wholesale_cost=[$11], ss_sales_price=[$13])
-                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($2))])
-                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                        HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                          HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2))])
-                            HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                    HiveAntiJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[anti])
+                      HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_ticket_number=[$9], ss_quantity=[$10], ss_wholesale_cost=[$11], ss_sales_price=[$13])
+                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($2))])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                        HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2))])
+                          HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
             HiveProject($f1=[$0], $f2=[$1], $f2_0=[$2], $f3=[$3], $f4=[$4])
               HiveFilter(condition=[>($2, 0)])
                 HiveAggregate(group=[{2, 3}], agg#0=[sum($4)], agg#1=[sum($5)], agg#2=[sum($6)])
@@ -158,14 +157,13 @@ HiveSortLimit(fetch=[100])
                       HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
                         HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                     HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$1], ws_bill_customer_sk=[$2], ws_quantity=[$4], ws_wholesale_cost=[$5], ws_sales_price=[$6])
-                      HiveFilter(condition=[IS NULL($8)])
-                        HiveJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[left], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_order_number=[$17], ws_quantity=[$18], ws_wholesale_cost=[$19], ws_sales_price=[$21])
-                            HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0), IS NOT NULL($3))])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(wr_item_sk=[$2], wr_order_number=[$13])
-                            HiveFilter(condition=[AND(IS NOT NULL($13), IS NOT NULL($2))])
-                              HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
+                      HiveAntiJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[anti])
+                        HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_order_number=[$17], ws_quantity=[$18], ws_wholesale_cost=[$19], ws_sales_price=[$21])
+                          HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0), IS NOT NULL($3))])
+                            HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                        HiveProject(wr_item_sk=[$2], wr_order_number=[$13])
+                          HiveFilter(condition=[AND(IS NOT NULL($13), IS NOT NULL($2))])
+                            HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
           HiveProject($f2=[$1], $f3=[$2], IS NOT NULL=[IS NOT NULL($2)], CASE=[CASE(IS NOT NULL($2), $2, 0:BIGINT)], CASE7=[CASE(IS NOT NULL($3), $3, 0:DECIMAL(17, 2))], CASE8=[CASE(IS NOT NULL($4), $4, 0:DECIMAL(17, 2))])
             HiveFilter(condition=[>($2, 0)])
               HiveProject(cs_item_sk=[$1], cs_bill_customer_sk=[$0], $f2=[$2], $f3=[$3], $f4=[$4])
@@ -175,12 +173,11 @@ HiveSortLimit(fetch=[100])
                       HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
                         HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                     HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$1], cs_item_sk=[$2], cs_quantity=[$4], cs_wholesale_cost=[$5], cs_sales_price=[$6])
-                      HiveFilter(condition=[IS NULL($8)])
-                        HiveJoin(condition=[AND(=($8, $3), =($2, $7))], joinType=[left], algorithm=[none], cost=[not available])
-                          HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_order_number=[$17], cs_quantity=[$18], cs_wholesale_cost=[$19], cs_sales_price=[$21])
-                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($15))])
-                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                          HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
-                            HiveFilter(condition=[AND(IS NOT NULL($16), IS NOT NULL($2))])
-                              HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+                      HiveAntiJoin(condition=[AND(=($8, $3), =($2, $7))], joinType=[anti])
+                        HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_order_number=[$17], cs_quantity=[$18], cs_wholesale_cost=[$19], cs_sales_price=[$21])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($15))])
+                            HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                        HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
+                          HiveFilter(condition=[AND(IS NOT NULL($16), IS NOT NULL($2))])
+                            HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
index b8521f4..722de27 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
@@ -68,30 +68,28 @@ POSTHOOK: Input: default@web_site
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
-  HiveFilter(condition=[IS NULL($13)])
-    HiveJoin(condition=[=($4, $14)], joinType=[left], algorithm=[none], cost=[not available])
-      HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
-        HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$3], ws_web_site_sk=[$4], ws_warehouse_sk=[$5], ws_order_number=[$6], ws_ext_ship_cost=[$7], ws_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], web_site_sk=[$11], web_company_name=[$12])
-          HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'TX':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
-                HiveFilter(condition=[AND(=($8, _UTF-16LE'TX'), IS NOT NULL($0))])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-              HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_warehouse_sk=[$15], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
-                  HiveFilter(condition=[AND(IS NOT NULL($13), IS NOT NULL($11), IS NOT NULL($2), IS NOT NULL($17))])
-                    HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
-                HiveProject(d_date_sk=[$0], d_date=[$2])
-                  HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00:TIMESTAMP(9), 1999-06-30 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-            HiveProject(web_site_sk=[$0], web_company_name=[CAST(_UTF-16LE'pri':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
-              HiveFilter(condition=[AND(=($14, _UTF-16LE'pri'), IS NOT NULL($0))])
-                HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
-        HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
-          HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($17))])
-            HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
-      HiveProject(literalTrue=[true], wr_order_number0=[$0])
-        HiveAggregate(group=[{13}])
-          HiveFilter(condition=[IS NOT NULL($13)])
-            HiveTableScan(table=[[default, web_returns]], table:alias=[wr1])
+  HiveAntiJoin(condition=[=($4, $14)], joinType=[anti])
+    HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
+      HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$3], ws_web_site_sk=[$4], ws_warehouse_sk=[$5], ws_order_number=[$6], ws_ext_ship_cost=[$7], ws_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], web_site_sk=[$11], web_company_name=[$12])
+        HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'TX':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
+              HiveFilter(condition=[AND(=($8, _UTF-16LE'TX'), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+            HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_warehouse_sk=[$15], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
+                HiveFilter(condition=[AND(IS NOT NULL($13), IS NOT NULL($11), IS NOT NULL($2), IS NOT NULL($17))])
+                  HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+              HiveProject(d_date_sk=[$0], d_date=[$2])
+                HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00:TIMESTAMP(9), 1999-06-30 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(web_site_sk=[$0], web_company_name=[CAST(_UTF-16LE'pri':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
+            HiveFilter(condition=[AND(=($14, _UTF-16LE'pri'), IS NOT NULL($0))])
+              HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
+      HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+        HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($17))])
+          HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
+    HiveProject(literalTrue=[true], wr_order_number0=[$13])
+      HiveFilter(condition=[IS NOT NULL($13)])
+        HiveTableScan(table=[[default, web_returns]], table:alias=[wr1])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
index 376b60a..0a3ef38 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
@@ -72,29 +72,27 @@ POSTHOOK: Input: default@date_dim
 POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
-  HiveFilter(condition=[IS NULL($13)])
-    HiveJoin(condition=[=($4, $14)], joinType=[left], algorithm=[none], cost=[not available])
-      HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
-        HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
-          HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
-            HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
-              HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
-                HiveFilter(condition=[=($8, _UTF-16LE'NY')])
-                  HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
-              HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
-                HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
-                  HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($10))])
-                    HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
-                HiveProject(d_date_sk=[$0], d_date=[$2])
-                  HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00:TIMESTAMP(9), 2001-05-31 00:00:00:TIMESTAMP(9))])
-                    HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-            HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
-              HiveFilter(condition=[IN($25, _UTF-16LE'Ziebach County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Levy County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Huron County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Franklin Parish':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Daviess County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
-                HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
-        HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
-          HiveFilter(condition=[IS NOT NULL($14)])
-            HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
-      HiveProject(literalTrue=[true], cr_order_number0=[$0])
-        HiveAggregate(group=[{16}])
-          HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
+  HiveAntiJoin(condition=[=($4, $14)], joinType=[anti])
+    HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
+      HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
+        HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+          HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
+              HiveFilter(condition=[=($8, _UTF-16LE'NY')])
+                HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+            HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+              HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
+                HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($10))])
+                  HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
+              HiveProject(d_date_sk=[$0], d_date=[$2])
+                HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00:TIMESTAMP(9), 2001-05-31 00:00:00:TIMESTAMP(9))])
+                  HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+          HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
+            HiveFilter(condition=[IN($25, _UTF-16LE'Ziebach County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Levy County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Huron County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Franklin Parish':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Daviess County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
+              HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
+      HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
+        HiveFilter(condition=[IS NOT NULL($14)])
+          HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
+    HiveProject(literalTrue=[true], cr_order_number0=[$16])
+      HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query69.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query69.q.out
index 761532c..c39199e 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query69.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query69.q.out
@@ -109,48 +109,45 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
 CBO PLAN:
 HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$4], sort4=[$6], dir0=[ASC], dir1=[ASC], dir2=[ASC], dir3=[ASC], dir4=[ASC], fetch=[100])
   HiveProject(cd_gender=[$0], cd_marital_status=[$1], cd_education_status=[$2], cnt1=[$5], cd_purchase_estimate=[$3], cnt2=[$5], cd_credit_rating=[$4], cnt3=[$5])
-    HiveAggregate(group=[{10, 11, 12, 13, 14}], agg#0=[count()])
-      HiveJoin(condition=[=($9, $1)], joinType=[inner], algorithm=[none], cost=[not available])
-        HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], literalTrue=[$5], ws_bill_customer_sk0=[$6], literalTrue0=[$7], cs_ship_customer_sk0=[$8])
-          HiveFilter(condition=[IS NULL($7)])
-            HiveJoin(condition=[=($0, $8)], joinType=[left], algorithm=[none], cost=[not available])
-              HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], literalTrue=[$5], ws_bill_customer_sk0=[$6])
-                HiveFilter(condition=[IS NULL($5)])
-                  HiveJoin(condition=[=($0, $6)], joinType=[left], algorithm=[none], cost=[not available])
-                    HiveSemiJoin(condition=[=($0, $5)], joinType=[semi])
-                      HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
-                        HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_addr_sk=[$4])
-                          HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($4))])
-                            HiveTableScan(table=[[default, customer]], table:alias=[c])
-                        HiveProject(ca_address_sk=[$0], ca_state=[$8])
-                          HiveFilter(condition=[IN($8, _UTF-16LE'CO':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'IL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'MN':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
-                            HiveTableScan(table=[[default, customer_address]], table:alias=[ca])
-                      HiveProject(ss_customer_sk0=[$1])
-                        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
-                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
-                              HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-                    HiveProject(literalTrue=[true], ws_bill_customer_sk0=[$0])
-                      HiveAggregate(group=[{1}])
-                        HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                          HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
-                            HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
-                              HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                          HiveProject(d_date_sk=[$0])
-                            HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
-                              HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-              HiveProject(literalTrue=[true], cs_ship_customer_sk0=[$0])
-                HiveAggregate(group=[{1}])
-                  HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
-                    HiveProject(cs_sold_date_sk=[$0], cs_ship_customer_sk=[$7])
-                      HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0))])
-                        HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                    HiveProject(d_date_sk=[$0])
-                      HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
-                        HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
-        HiveProject(cd_demo_sk=[$0], cd_gender=[$1], cd_marital_status=[$2], cd_education_status=[$3], cd_purchase_estimate=[$4], cd_credit_rating=[$5])
-          HiveTableScan(table=[[default, customer_demographics]], table:alias=[customer_demographics])
+    HiveAggregate(group=[{6, 7, 8, 9, 10}], agg#0=[count()])
+      HiveAntiJoin(condition=[=($0, $14)], joinType=[anti])
+        HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], cd_demo_sk=[$7], cd_gender=[$8], cd_marital_status=[$9], cd_education_status=[$10], cd_purchase_estimate=[$11], cd_credit_rating=[$12], literalTrue=[$5], ws_bill_customer_sk0=[$6])
+          HiveJoin(condition=[=($7, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], literalTrue=[$5], ws_bill_customer_sk0=[$6])
+              HiveFilter(condition=[IS NULL($5)])
+                HiveJoin(condition=[=($0, $6)], joinType=[left], algorithm=[none], cost=[not available])
+                  HiveSemiJoin(condition=[=($0, $5)], joinType=[semi])
+                    HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_addr_sk=[$4])
+                        HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($4))])
+                          HiveTableScan(table=[[default, customer]], table:alias=[c])
+                      HiveProject(ca_address_sk=[$0], ca_state=[$8])
+                        HiveFilter(condition=[IN($8, _UTF-16LE'CO':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'IL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'MN':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
+                          HiveTableScan(table=[[default, customer_address]], table:alias=[ca])
+                    HiveProject(ss_customer_sk0=[$1])
+                      HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                        HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
+                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                        HiveProject(d_date_sk=[$0])
+                          HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
+                            HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+                  HiveProject(literalTrue=[true], ws_bill_customer_sk0=[$1])
+                    HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+                      HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
+                        HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
+                          HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                      HiveProject(d_date_sk=[$0])
+                        HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
+                          HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+            HiveProject(cd_demo_sk=[$0], cd_gender=[$1], cd_marital_status=[$2], cd_education_status=[$3], cd_purchase_estimate=[$4], cd_credit_rating=[$5])
+              HiveTableScan(table=[[default, customer_demographics]], table:alias=[customer_demographics])
+        HiveProject(literalTrue=[true], cs_ship_customer_sk0=[$1])
+          HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+            HiveProject(cs_sold_date_sk=[$0], cs_ship_customer_sk=[$7])
+              HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+            HiveProject(d_date_sk=[$0])
+              HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
+                HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query78.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query78.q.out
index 14028ff..b893ae7 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query78.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query78.q.out
@@ -142,13 +142,12 @@ HiveSortLimit(fetch=[100])
                     HiveFilter(condition=[=($6, 2000)])
                       HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                   HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_quantity=[$4], ss_wholesale_cost=[$5], ss_sales_price=[$6])
-                    HiveFilter(condition=[IS NULL($8)])
-                      HiveJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[left], algorithm=[none], cost=[not available])
-                        HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_ticket_number=[$9], ss_quantity=[$10], ss_wholesale_cost=[$11], ss_sales_price=[$13])
-                          HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
-                            HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
-                        HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
-                          HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+                    HiveAntiJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[anti])
+                      HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_ticket_number=[$9], ss_quantity=[$10], ss_wholesale_cost=[$11], ss_sales_price=[$13])
+                        HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+                          HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+                      HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+                        HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
             HiveProject($f2=[$1], $f3=[$2], $f4=[$3], $f5=[$4])
               HiveFilter(condition=[>($2, 0)])
                 HiveProject(cs_item_sk=[$1], cs_bill_customer_sk=[$0], $f2=[$2], $f3=[$3], $f4=[$4])
@@ -158,13 +157,12 @@ HiveSortLimit(fetch=[100])
                         HiveFilter(condition=[=($6, 2000)])
                           HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                       HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$1], cs_item_sk=[$2], cs_quantity=[$4], cs_wholesale_cost=[$5], cs_sales_price=[$6])
-                        HiveFilter(condition=[IS NULL($8)])
-                          HiveJoin(condition=[AND(=($8, $3), =($2, $7))], joinType=[left], algorithm=[none], cost=[not available])
-                            HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_order_number=[$17], cs_quantity=[$18], cs_wholesale_cost=[$19], cs_sales_price=[$21])
-                              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
-                                HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
-                            HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
-                              HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+                        HiveAntiJoin(condition=[AND(=($8, $3), =($2, $7))], joinType=[anti])
+                          HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_order_number=[$17], cs_quantity=[$18], cs_wholesale_cost=[$19], cs_sales_price=[$21])
+                            HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+                              HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+                          HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
+                            HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
           HiveProject($f1=[$0], $f2=[$1], $f2_0=[$2], $f3=[$3], $f4=[$4])
             HiveFilter(condition=[>($2, 0)])
               HiveAggregate(group=[{2, 3}], agg#0=[sum($4)], agg#1=[sum($5)], agg#2=[sum($6)])
@@ -173,11 +171,10 @@ HiveSortLimit(fetch=[100])
                     HiveFilter(condition=[=($6, 2000)])
                       HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
                   HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$1], ws_bill_customer_sk=[$2], ws_quantity=[$4], ws_wholesale_cost=[$5], ws_sales_price=[$6])
-                    HiveFilter(condition=[IS NULL($8)])
-                      HiveJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[left], algorithm=[none], cost=[not available])
-                        HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_order_number=[$17], ws_quantity=[$18], ws_wholesale_cost=[$19], ws_sales_price=[$21])
-                          HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
-                            HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
-                        HiveProject(wr_item_sk=[$2], wr_order_number=[$13])
-                          HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
+                    HiveAntiJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[anti])
+                      HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_order_number=[$17], ws_quantity=[$18], ws_wholesale_cost=[$19], ws_sales_price=[$21])
+                        HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
+                          HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+                      HiveProject(wr_item_sk=[$2], wr_order_number=[$13])
+                        HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
 
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
index ef108a6..8210cec 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
@@ -68,29 +68,27 @@ POSTHOOK: Input: default@web_site
... 2593 lines suppressed ...