HIVE-24957 HIVE-24999: Inefficient & wrong CBO plans in the presence of subqueries (Stamatis Zampetakis, reviewed by Krisztian Kasa)

* HIVE-24999: HiveSubQueryRemoveRule generates invalid plan for IN subquery with correlations

1. Add workaround for CALCITE-4574 in HiveRelBuilder to avoid generating
invalid plans (filter with references to columns which do not exist).

2. Adapt HiveRelDecorrelator based on new plans generated by HiveSubQueryRemoveRule

2a. Remove workaround getNewForOldInputRef that was needed due to the
invalid plans.

2b. Adapt input references based on new the input operator (frame) inside
decorrelateInputWithValueGenerator method.

3c. Refactor DecorrelateRexShuttle#visitCall to improve readability and
cover a few more corner cases.

3. Add subquery_in_invalid_intermediate_plan.q with problematic plans
relevant for the case.

4. Add CBO explain plans in queries related to masking since there are
easier to read and compare with. There are few plan regressions that
will be fixed by HIVE-24957.

* HIVE-24957: Wrong results when subquery has COALESCE in correlation predicate

1. Add plan transformations before starting the core RelDecorrelator logic
to bring the plan into an equivalent but more convenient form that can be
decorrelated into more efficient and correct plans.

2. Adapt HiveRelDecorrelator#decorrelateInputWithValueGenerator to avoid
creating value generator for already satisfied correlations present in
the input.

3. Based on the changes above many plans with subqueries become more
efficient since the value generator is no longer necessary and it is dropped.

4. Add subquery_complex_correlation_predicates.q which includes queries
generating wrong results without the new transformations.

5. Add CBO plans in few queries since they are easier to read and reason
about correctness and efficiency.
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java
index 9bf7f95..27d0070 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/Bug.java
@@ -31,6 +31,11 @@
 public final class Bug {
 
   /**
+   * Whether <a href="https://issues.apache.org/jira/browse/CALCITE-1851">CALCITE-1851</a> is fixed.
+   */
+  public static final boolean CALCITE_1851_FIXED = false;
+  
+  /**
    * Whether <a href="https://issues.apache.org/jira/browse/CALCITE-3982">issue
    * CALCITE-3982</a> is fixed.
    */
@@ -47,4 +52,9 @@
    * CALCITE-4499</a> is fixed.
    */
   public static final boolean CALCITE_4499_FIXED = false;
+
+  /**
+   * Whether <a href="https://issues.apache.org/jira/browse/CALCITE-4574">CALCITE-4574</a> is fixed.
+   */
+  public static final boolean CALCITE_4574_FIXED = false;
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
index 4e184dc..9f521a7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelBuilder.java
@@ -17,14 +17,24 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite;
 
+import com.google.common.collect.Iterables;
 import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.server.CalciteServerStatement;
@@ -35,6 +45,8 @@
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveMergeableAggregate;
 import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlCountAggFunction;
 import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlMinMaxAggFunction;
@@ -44,6 +56,10 @@
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
 /**
  * Builder for relational expressions in Hive.
  *
@@ -158,6 +174,89 @@
     return null;
   }
 
+  /** Creates a {@link Join} with correlating variables. */
+  @Override
+  public RelBuilder join(JoinRelType joinType, RexNode condition,
+      Set<CorrelationId> variablesSet) {
+    if (Bug.CALCITE_4574_FIXED) {
+      throw new IllegalStateException("Method overriding should be removed once CALCITE-4574 is fixed");
+    }
+    RelNode right = this.peek(0);
+    RelNode left = this.peek(1);
+    final boolean correlate = variablesSet.size() == 1;
+    RexNode postCondition = literal(true);
+    if (correlate) {
+      final CorrelationId id = Iterables.getOnlyElement(variablesSet);
+      if (!RelOptUtil.notContainsCorrelation(left, id, Litmus.IGNORE)) {
+        throw new IllegalArgumentException("variable " + id
+            + " must not be used by left input to correlation");
+      }
+      // Correlate does not have an ON clause.
+      switch (joinType) {
+      case LEFT:
+      case SEMI:
+      case ANTI:
+        // For a LEFT/SEMI/ANTI, predicate must be evaluated first.
+        filter(condition.accept(new Shifter(left, id, right)));
+        right = this.peek(0);
+        break;
+      case INNER:
+        // For INNER, we can defer.
+        postCondition = condition;
+        break;
+      default:
+        throw new IllegalArgumentException("Correlated " + joinType + " join is not supported");
+      }
+      final ImmutableBitSet requiredColumns = RelOptUtil.correlationColumns(id, right);
+      List<RexNode> leftFields = this.fields(2, 0);
+      List<RexNode> requiredFields = new ArrayList<>();
+      for (int i = 0; i < leftFields.size(); i++) {
+        if (requiredColumns.get(i)) {
+          requiredFields.add(leftFields.get(i));
+        }
+      }
+      correlate(joinType, id, requiredFields);
+      filter(postCondition);
+    } else {
+      // When there is no correlation use the default logic which works OK for now
+      // Cannot copy-paste the respective code here cause we don't have access to stack,
+      // Frame etc. and we might lose existing aliases in the builder
+      assert variablesSet.isEmpty();
+      super.join(joinType,condition, variablesSet);
+    }
+    return this;
+  }
+
+  /** Shuttle that shifts a predicate's inputs to the left, replacing early
+   * ones with references to a
+   * {@link RexCorrelVariable}. */
+  private class Shifter extends RexShuttle {
+    private final RelNode left;
+    private final CorrelationId id;
+    private final RelNode right;
+
+    Shifter(RelNode left, CorrelationId id, RelNode right) {
+      this.left = left;
+      this.id = id;
+      this.right = right;
+      if (Bug.CALCITE_4574_FIXED) {
+        throw new IllegalStateException("Class should be redundant once CALCITE-4574 is fixed");
+      }
+    }
+
+    public RexNode visitInputRef(RexInputRef inputRef) {
+      final RelDataType leftRowType = left.getRowType();
+      final RexBuilder rexBuilder = getRexBuilder();
+      final int leftCount = leftRowType.getFieldCount();
+      if (inputRef.getIndex() < leftCount) {
+        final RexNode v = rexBuilder.makeCorrel(leftRowType, id);
+        return rexBuilder.makeFieldAccess(v, inputRef.getIndex());
+      } else {
+        return rexBuilder.makeInputRef(right, inputRef.getIndex() - leftCount);
+      }
+    }
+  }
+
   @Override
   protected boolean shouldMergeProject() {
     /* CALCITE-2470 added ability to merge Project-s together.
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/CorrelateProjectExtractor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/CorrelateProjectExtractor.java
new file mode 100644
index 0000000..a2af599
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/CorrelateProjectExtractor.java
@@ -0,0 +1,345 @@
+/*
+ * 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.rel.RelHomogeneousShuttle;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A visitor for relational expressions that extracts a {@link org.apache.calcite.rel.core.Project}, with a "simple"
+ * computation over the correlated variables, from the right side of a correlation
+ * ({@link org.apache.calcite.rel.core.Correlate}) and places it on the left side.
+ *
+ * <h3>Plan before</h3>
+ * <pre>
+ * LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{7}])
+ *   LogicalTableScan(table=[[scott, EMP]])
+ *   LogicalFilter(condition=[=($0, +(10, $cor0.DEPTNO))])
+ *     LogicalTableScan(table=[[scott, DEPT]])
+ * </pre>
+ *
+ * <h3>Plan after</h3>
+ * <pre>
+ * LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3],... DNAME=[$10], LOC=[$11])
+ *   LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{8}])
+ *     LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], ... COMM=[$6], DEPTNO=[$7], $f8=[+(10, $7)])
+ *       LogicalTableScan(table=[[scott, EMP]])
+ *     LogicalFilter(condition=[=($0, $cor0.$f8)])
+ *       LogicalTableScan(table=[[scott, DEPT]])
+ * </pre>
+ *
+ * Essentially this transformation moves the computation over a correlated expression from the inner
+ * loop to the outer loop. It materializes the computation on the left side and flattens expressions
+ * on correlated variables on the right side.
+ */
+public final class CorrelateProjectExtractor extends RelHomogeneousShuttle {
+
+  private final RelBuilderFactory builderFactory;
+
+  public CorrelateProjectExtractor(RelBuilderFactory factory) {
+    this.builderFactory = factory;
+  }
+
+  @Override public RelNode visit(LogicalCorrelate correlate) {
+    RelNode left = correlate.getLeft().accept(this);
+    RelNode right = correlate.getRight().accept(this);
+    int oldLeft = left.getRowType().getFieldCount();
+    // Find the correlated expressions from the right side that can be moved to the left
+    Set<RexNode> callsWithCorrelationInRight =
+        findCorrelationDependentCalls(correlate.getCorrelationId(), right);
+    boolean isTrivialCorrelation =
+        callsWithCorrelationInRight.stream().allMatch(exp -> exp instanceof RexFieldAccess);
+    // Early exit condition
+    if (isTrivialCorrelation) {
+      if (correlate.getLeft().equals(left) && correlate.getRight().equals(right)) {
+        return correlate;
+      } else {
+        return correlate.copy(
+            correlate.getTraitSet(),
+            left,
+            right,
+            correlate.getCorrelationId(),
+            correlate.getRequiredColumns(),
+            correlate.getJoinType());
+      }
+    }
+
+    RelBuilder builder = builderFactory.create(correlate.getCluster(), null);
+    // Transform the correlated expression from the right side to an expression over the left side
+    builder.push(left);
+
+    List<RexNode> callsWithCorrelationOverLeft = new ArrayList<>();
+    for (RexNode callInRight : callsWithCorrelationInRight) {
+      callsWithCorrelationOverLeft.add(replaceCorrelationsWithInputRef(callInRight, builder));
+    }
+    builder.projectPlus(callsWithCorrelationOverLeft);
+
+    // Construct the mapping to transform the expressions in the right side based on the new
+    // projection in the left side.
+    Map<RexNode, RexNode> transformMapping = new HashMap<>();
+    for (RexNode callInRight : callsWithCorrelationInRight) {
+      RexBuilder xb = builder.getRexBuilder();
+      RexNode v = xb.makeCorrel(builder.peek().getRowType(), correlate.getCorrelationId());
+      RexNode flatCorrelationInRight = xb.makeFieldAccess(v, oldLeft + transformMapping.size());
+      transformMapping.put(callInRight, flatCorrelationInRight);
+    }
+
+    // Select the required fields/columns from the left side of the correlation. Based on the code
+    // above all these fields should be at the end of the left relational expression.
+    List<RexNode> requiredFields =
+        builder.fields(
+            ImmutableBitSet.range(oldLeft, oldLeft + callsWithCorrelationOverLeft.size()).asList());
+
+    final int newLeft = builder.fields().size();
+    // Transform the expressions in the right side using the mapping constructed earlier.
+    right = replaceExpressionsUsingMap(right, transformMapping);
+    builder.push(right);
+
+    builder.correlate(correlate.getJoinType(), correlate.getCorrelationId(), requiredFields);
+    // Remove the additional fields that were added for the needs of the correlation to keep the old
+    // and new plan equivalent.
+    List<Integer> retainFields;
+    switch (correlate.getJoinType()) {
+    case SEMI:
+    case ANTI:
+      retainFields = ImmutableBitSet.range(0, oldLeft).asList();
+      break;
+    case LEFT:
+    case INNER:
+      retainFields =
+          ImmutableBitSet.builder()
+              .set(0, oldLeft)
+              .set(newLeft, newLeft + right.getRowType().getFieldCount())
+              .build()
+              .asList();
+      break;
+    default:
+      throw new AssertionError(correlate.getJoinType());
+    }
+    builder.project(builder.fields(retainFields));
+    return builder.build();
+  }
+
+  /**
+   * Traverses a plan and finds all simply correlated row expressions with the specified id.
+   */
+  private static Set<RexNode> findCorrelationDependentCalls(CorrelationId corrId, RelNode plan) {
+    SimpleCorrelationCollector finder = new SimpleCorrelationCollector(corrId);
+    plan.accept(new RelHomogeneousShuttle() {
+      @Override public RelNode visit(RelNode other) {
+        if (other instanceof Project || other instanceof Filter) {
+          other.accept(finder);
+        }
+        return super.visit(other);
+      }
+    });
+    return finder.correlations;
+  }
+
+  /**
+   * Replaces all row expressions in the plan using the provided mapping.
+   *
+   * @param plan the relational expression on which we want to perform the replacements.
+   * @param mapping a mapping defining how to replace row expressions in the plan
+   * @return a new relational expression where all expressions present in the mapping are replaced.
+   */
+  private static RelNode replaceExpressionsUsingMap(RelNode plan, Map<RexNode, RexNode> mapping) {
+    CallReplacer replacer = new CallReplacer(mapping);
+    return plan.accept(new RelHomogeneousShuttle() {
+      @Override public RelNode visit(RelNode other) {
+        RelNode mNode = super.visitChildren(other);
+        return mNode.accept(replacer);
+      }
+    });
+  }
+
+  /**
+   * A collector of simply correlated row expressions.
+   *
+   * The shuttle traverses the tree and collects all calls and field accesses that are classified
+   * as simply correlated expressions. Multiple nodes in a call hierarchy may satisfy the criteria
+   * of a simple correlation so we peek the expressions closest to the root.
+   *
+   * @see SimpleCorrelationDetector
+   */
+  private static final class SimpleCorrelationCollector extends RexShuttle {
+    private final CorrelationId correlationId;
+    // Clients are iterating over the collection thus it is better to use LinkedHashSet to keep
+    // plans stable among executions.
+    private final Set<RexNode> correlations = new LinkedHashSet<>();
+
+    SimpleCorrelationCollector(CorrelationId corrId) {
+      this.correlationId = corrId;
+    }
+
+    @Override public RexNode visitCall(RexCall call) {
+      if (isSimpleCorrelatedExpression(call, correlationId)) {
+        correlations.add(call);
+        return call;
+      } else {
+        return super.visitCall(call);
+      }
+    }
+
+    @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+      if (isSimpleCorrelatedExpression(fieldAccess, correlationId)) {
+        correlations.add(fieldAccess);
+        return fieldAccess;
+      } else {
+        return super.visitFieldAccess(fieldAccess);
+      }
+    }
+  }
+
+  /**
+   * Returns whether the specified node is a simply correlated expression.
+   */
+  private static boolean isSimpleCorrelatedExpression(RexNode node, CorrelationId id) {
+    Boolean r = node.accept(new SimpleCorrelationDetector(id));
+    return r == null ? Boolean.FALSE : r;
+  }
+
+  /**
+   * A visitor classifying row expressions as simply correlated if they satisfy the conditions
+   * below.
+   * <ul>
+   * <li>all correlated variables have the specified correlation id</li>
+   * <li>all leafs are either correlated variables, dynamic parameters, or literals</li>
+   * <li>intermediate nodes are either calls or field access expressions</li>
+   * </ul>
+   *
+   * Examples:
+   * <pre>
+   * +(10, $cor0.DEPTNO) -> TRUE
+   * /(100,+(10, $cor0.DEPTNO)) -> TRUE
+   * CAST(+(10, $cor0.DEPTNO)):INTEGER NOT NULL -> TRUE
+   * +($0, $cor0.DEPTNO) -> FALSE
+   * </pre>
+   *
+   */
+  private static class SimpleCorrelationDetector extends RexVisitorImpl<Boolean> {
+    private final CorrelationId corrId;
+
+    private SimpleCorrelationDetector(CorrelationId corrId) {
+      super(true);
+      this.corrId = corrId;
+    }
+
+    @Override public Boolean visitOver(RexOver over) {
+      return Boolean.FALSE;
+    }
+
+    @Override public Boolean visitSubQuery(RexSubQuery subQuery) {
+      return Boolean.FALSE;
+    }
+
+    @Override public Boolean visitCall(RexCall call) {
+      Boolean hasSimpleCorrelation = null;
+      for (RexNode op : call.operands) {
+        Boolean b = op.accept(this);
+        if (b != null) {
+          hasSimpleCorrelation = hasSimpleCorrelation == null ? b : hasSimpleCorrelation && b;
+        }
+      }
+      return hasSimpleCorrelation == null ? Boolean.FALSE : hasSimpleCorrelation;
+    }
+
+    @Override public Boolean visitFieldAccess(RexFieldAccess fieldAccess) {
+      return fieldAccess.getReferenceExpr().accept(this);
+    }
+
+    @Override public Boolean visitInputRef(RexInputRef inputRef) {
+      return Boolean.FALSE;
+    }
+
+    @Override public Boolean visitCorrelVariable(RexCorrelVariable correlVariable) {
+      return correlVariable.id.equals(corrId);
+    }
+
+    @Override public Boolean visitTableInputRef(RexTableInputRef ref) {
+      return Boolean.FALSE;
+    }
+
+    @Override public Boolean visitLocalRef(RexLocalRef localRef) {
+      return Boolean.FALSE;
+    }
+
+    @Override public Boolean visitPatternFieldRef(RexPatternFieldRef fieldRef) {
+      return Boolean.FALSE;
+    }
+  }
+
+  private static RexNode replaceCorrelationsWithInputRef(RexNode exp, RelBuilder b) {
+    return exp.accept(new RexShuttle() {
+      @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+        if (fieldAccess.getReferenceExpr() instanceof RexCorrelVariable) {
+          return b.field(fieldAccess.getField().getIndex());
+        }
+        return super.visitFieldAccess(fieldAccess);
+      }
+    });
+  }
+
+  /**
+   * A visitor traversing row expressions and replacing calls with other expressions according
+   * to the specified mapping.
+   */
+  private static final class CallReplacer extends RexShuttle {
+    private final Map<RexNode, RexNode> mapping;
+
+    CallReplacer(Map<RexNode, RexNode> mapping) {
+      this.mapping = mapping;
+    }
+
+    @Override public RexNode visitCall(RexCall oldCall) {
+      RexNode newCall = mapping.get(oldCall);
+      if (newCall != null) {
+        return newCall;
+      } else {
+        return super.visitCall(oldCall);
+      }
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/FilterFlattenCorrelatedConditionRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/FilterFlattenCorrelatedConditionRule.java
new file mode 100644
index 0000000..ba78c8f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/FilterFlattenCorrelatedConditionRule.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Planner rule that matches a {@link Filter} expression with correlated variables, and rewrites the
+ * condition in a simpler form that is more convenient for the decorrelation logic.
+ *
+ * Uncorrelated calls below a comparison operator are turned into input references by extracting the
+ * computation in a {@link org.apache.calcite.rel.core.Project} expression. An additional projection
+ * may be added on top of the new filter to retain expression equivalence.
+ *
+ * <h3>Sub-plan before</h3>
+ * <pre>
+ * LogicalProject($f0=[true])
+ *   LogicalFilter(condition=[=($cor0.DEPTNO, +($7, 30))])
+ *     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+ * </pre>
+ * <h3>Sub-plan after</h3>
+ * <pre>
+ * LogicalProject($f0=[true])
+ *   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2],..., COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+ *     LogicalFilter(condition=[=($cor0.DEPTNO, $9)])
+ *       LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2],..., SLACKER=[$8], $f9=[+($7, 30)])
+ *         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+ * </pre>
+ */
+public final class FilterFlattenCorrelatedConditionRule
+    extends RelOptRule {
+
+  public FilterFlattenCorrelatedConditionRule(Class<? extends Filter> clazz, RelBuilderFactory factory,
+      String description) {
+    super(operand(clazz, any()), factory, description);
+  }
+
+  @Override public boolean matches(RelOptRuleCall call) {
+    Filter filter = call.rel(0);
+    return RexUtil.containsCorrelation(filter.getCondition());
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    Filter filter = call.rel(0);
+    RelBuilder b = call.builder();
+    b.push(filter.getInput());
+    final int proj = b.fields().size();
+    List<RexNode> projOperands = new ArrayList<>();
+    RexNode newCondition = filter.getCondition().accept(new RexShuttle() {
+      @Override public RexNode visitCall(RexCall call) {
+        switch (call.getKind()) {
+        case EQUALS:
+        case NOT_EQUALS:
+        case GREATER_THAN:
+        case GREATER_THAN_OR_EQUAL:
+        case LESS_THAN:
+        case LESS_THAN_OR_EQUAL:
+          RexNode op0 = call.operands.get(0);
+          RexNode op1 = call.operands.get(1);
+          final int replaceIndex;
+          if (RexUtil.containsCorrelation(op1) && isUncorrelatedCall(op0)) {
+            replaceIndex = 0;
+          } else if (RexUtil.containsCorrelation(op0) && isUncorrelatedCall(op1)) {
+            replaceIndex = 1;
+          } else {
+            // Structure does not match, do not replace
+            replaceIndex = -1;
+          }
+          if (replaceIndex != -1) {
+            List<RexNode> copyOperands = new ArrayList<>(call.operands);
+            RexNode oldOp = call.operands.get(replaceIndex);
+            RexNode newOp = b.getRexBuilder()
+                .makeInputRef(oldOp.getType(), proj + projOperands.size());
+            projOperands.add(oldOp);
+            copyOperands.set(replaceIndex, newOp);
+            return call.clone(call.type, copyOperands);
+          }
+          break;
+        case AND:
+        case OR:
+          return super.visitCall(call);
+        }
+        return call;
+      }
+    });
+    if (newCondition.equals(filter.getCondition())) {
+      return;
+    }
+    b.projectPlus(projOperands);
+    b.filter(newCondition);
+    b.project(b.fields(ImmutableBitSet.range(proj).asList()));
+    call.transformTo(b.build());
+  }
+
+  private static boolean isUncorrelatedCall(RexNode node) {
+    return node instanceof RexCall && !RexUtil.containsCorrelation(node);
+  }
+}
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 bbd606c..e438487 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
@@ -84,7 +84,6 @@
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
@@ -94,6 +93,7 @@
 import org.apache.calcite.util.Stacks;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
+import org.apache.hadoop.hive.ql.optimizer.calcite.Bug;
 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;
@@ -154,7 +154,10 @@
 
   protected static final Logger LOG = LoggerFactory.getLogger(
           HiveRelDecorrelator.class);
-
+  
+  private static final FilterFlattenCorrelatedConditionRule FLATTEN_CORRELATED_CONDITION_RULE =
+      new FilterFlattenCorrelatedConditionRule(HiveFilter.class, HiveRelFactories.HIVE_BUILDER,
+          "HiveFilterFlattenCorrelatedConditionRule");
   //~ Instance fields --------------------------------------------------------
 
   private final RelBuilder relBuilder;
@@ -239,17 +242,25 @@
             .addRuleInstance(new AdjustProjectForCountAggregateRule(true))
             .addRuleInstance(HiveFilterJoinRule.FILTER_ON_JOIN)
             .addRuleInstance(HiveFilterProjectTransposeRule.INSTANCE)
+            .addRuleInstance(FLATTEN_CORRELATED_CONDITION_RULE)
             // FilterCorrelateRule rule mistakenly pushes a FILTER, consiting of correlated vars,
             // on top of LogicalCorrelate to within  left input for scalar corr queries
             // which causes exception during decorrelation. This has been disabled for now.
             //.addRuleInstance(FilterCorrelateRule.INSTANCE)
             .build();
-
     HepPlanner planner = createPlanner(program);
 
     planner.setRoot(root);
     root = planner.findBestExp();
-
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Plan before extracting correlated computations:\n" + RelOptUtil.toString(root));
+    }
+    root = root.accept(new CorrelateProjectExtractor(HiveRelFactories.HIVE_BUILDER));
+    // Necessary to update cm (CorrelMap) since CorrelateProjectExtractor above may modify the plan
+    this.cm = new CorelMapBuilder().build(root);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Plan after extracting correlated computations:\n" + RelOptUtil.toString(root));
+    }
     // Perform decorrelation.
     map.clear();
 
@@ -994,7 +1005,16 @@
       SortedMap<CorDef, Integer> coreMap = new TreeMap<>();
       for (CorRef correlation : corVarList) {
         final CorDef def = correlation.def();
-        if (corDefOutputs.containsKey(def) || coreMap.containsKey(def)) {
+        // If the correlation/variable is already provided by the input
+        // then we don't have to look for equivalent expressions and 
+        // we don't need to create value generator for them.
+        if (corDefOutputs.containsKey(def)) {
+          coreMap.put(def, corDefOutputs.get(def));
+          continue;
+        }
+        // If the correlation/variable is in the map then we already
+        // seen this before in this loop so we don't need to treat it again.
+        if (coreMap.containsKey(def)) {
           continue;
         }
         try {
@@ -1005,7 +1025,9 @@
           // is generated
           def.setPredicateKind((SqlOperator) ((Pair)((Pair)e.getNode()).getValue()).getKey());
           def.setIsLeft((boolean)((Pair)((Pair) e.getNode()).getValue()).getValue());
-          coreMap.put(def, (Integer)((Pair) e.getNode()).getKey());
+          final Integer oldInputRef = (Integer) ((Pair) e.getNode()).getKey();
+          final Integer newInputRef = frame.oldToNewOutputs.get(oldInputRef);
+          coreMap.put(def, newInputRef);
         }
       }
       // If all correlation variables are now satisfied, skip creating a value
@@ -1220,10 +1242,12 @@
 
     if (leftFrame == null || rightFrame == null) {
       // If any input has not been rewritten, do not rewrite this rel.
+      valueGen.pop();
       return null;
     }
 
     if (rightFrame.corDefOutputs.isEmpty()) {
+      valueGen.pop();
       return null;
     }
 
@@ -1415,17 +1439,6 @@
       oldOrdinal -= n;
     }
 
-    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 || 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());
-        }
-      }
-    }
-
     assert oldInput != null;
 
     final Frame frame = map.get(oldInput);
@@ -1690,46 +1703,49 @@
       this.valueGenerator = valueGenerator;
     }
 
-    // DecorrelateRexShuttle ends up decorrelating expressions cor.col1 <> $4
-    // to $4=$4 if value generator is not generated, $4<>$4 is further simplified
-    // to false. This is wrong and messes up the whole tree. To prevent this visitCall
-    // is overridden to rewrite/simply such predicates to is not null.
-    // we also need to take care that we do this only for correlated predicates and
-    // not user specified explicit predicates
-    // TODO:  This code should be removed once CALCITE-1851 is fixed and
-    // there is support of not equal
     @Override  public RexNode visitCall(final RexCall call) {
-      if(!valueGenerator) {
-        if(call.getOperands().size() == 2) {
-          final List<RexNode> operands = new ArrayList<>(call.operands);
-          RexNode o0 = operands.get(0);
-          RexNode o1 = operands.get(1);
-          boolean isCorrelated = false;
-          if (o0 instanceof RexFieldAccess && (cm.mapFieldAccessToCorRef.get(o0) != null)) {
-            o0 = decorrFieldAccess((RexFieldAccess) o0);
-            isCorrelated = true;
-
-          }
-          if (o1 instanceof RexFieldAccess && (cm.mapFieldAccessToCorRef.get(o1) != null)) {
-            o1 = decorrFieldAccess((RexFieldAccess) o1);
-            isCorrelated = true;
-          }
-          if (isCorrelated && o0.equals(o1)) {
-            return rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, o0);
-          }
-
-          final List<RexNode> newOperands = new ArrayList<>();
-          newOperands.add(o0);
-          newOperands.add(o1);
-          boolean[] update = {false};
-          List<RexNode> clonedOperands = visitList(newOperands, update);
-
-          return relBuilder.call(call.getOperator(), clonedOperands);
-        }
+      if (Bug.CALCITE_1851_FIXED) {
+        throw new AssertionError("Overriding should be removed");
       }
-      return super.visitCall(call);
+      if (valueGenerator) {
+        return super.visitCall(call);
+      }
+      switch (call.getKind()) {
+      case EQUALS:
+      case NOT_EQUALS:
+      case GREATER_THAN:
+      case GREATER_THAN_OR_EQUAL:
+      case LESS_THAN:
+      case LESS_THAN_OR_EQUAL:
+        // Expressions of the form OP(cor.col1, $4) are transformed to OP($4, $4) where OP in (<,<=,=,>,>=,<>).
+        // OP($4, $4) can be further simplified to TRUE/FALSE (e.g., $4 <> $4) that is wrong and messes up the tree.
+        // To prevent this simplifications we rewrite OP($4, $4) to IS_NOT_NULL($4).
+        // This should be done only for correlated predicates and not user specified explicit predicates  
+        if (hasCorrelation(call)) {
+          List<RexNode> ops = visitList(call.operands, new boolean[] { false });
+          assert ops.size() == 2;
+          RexNode o0 = ops.get(0);
+          RexNode o1 = ops.get(1);
+          if (o0.equals(o1)) {
+            return relBuilder.call(SqlStdOperatorTable.IS_NOT_NULL, o0);
+          } else {
+            return relBuilder.call(call.getOperator(), ops);
+          }
+        }
+      default:
+        return super.visitCall(call);
+      }
     }
 
+    private boolean hasCorrelation(RexCall call) {
+      for (RexNode n : call.operands) {
+        if (n instanceof RexFieldAccess && cm.mapFieldAccessToCorRef.containsKey(n)) {
+          return true;
+        }
+      }
+      return false;
+    }
+    
     @Override public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
       return decorrFieldAccess(fieldAccess);
     }
@@ -3001,7 +3017,6 @@
             new Supplier<TreeSet<CorRef>>() {
               @Override
               public TreeSet<CorRef> get() {
-                Bug.upgrade("use MultimapBuilder when we're on Guava-16");
                 return Sets.newTreeSet();
               }
             });
diff --git a/ql/src/test/queries/clientpositive/masking_12.q b/ql/src/test/queries/clientpositive/masking_12.q
index 9b99f0c..ae55446 100644
--- a/ql/src/test/queries/clientpositive/masking_12.q
+++ b/ql/src/test/queries/clientpositive/masking_12.q
@@ -16,6 +16,9 @@
 
 create view `v1_n9` as select * from `masking_test_subq_n1`;
 
+explain cbo
+select * from `v1_n9`
+limit 20;
 explain
 select * from `v1_n9`
 limit 20;
diff --git a/ql/src/test/queries/clientpositive/masking_3.q b/ql/src/test/queries/clientpositive/masking_3.q
index 012ce8b..8c2ccdb 100644
--- a/ql/src/test/queries/clientpositive/masking_3.q
+++ b/ql/src/test/queries/clientpositive/masking_3.q
@@ -8,24 +8,31 @@
 
 create table masking_test_subq_n3 as select cast(key as int) as key, value from src;
 
+explain cbo select * from masking_test_subq_n3;
 explain select * from masking_test_subq_n3;
 select * from masking_test_subq_n3;
 
+explain cbo select * from masking_test_subq_n3 where key > 0;
 explain select * from masking_test_subq_n3 where key > 0;
 select * from masking_test_subq_n3 where key > 0;
 
+explain cbo select key from masking_test_subq_n3 where key > 0;
 explain select key from masking_test_subq_n3 where key > 0;
 select key from masking_test_subq_n3 where key > 0;
 
+explain cbo select value from masking_test_subq_n3 where key > 0;
 explain select value from masking_test_subq_n3 where key > 0;
 select value from masking_test_subq_n3 where key > 0;
 
+explain cbo select * from masking_test_subq_n3 join srcpart on (masking_test_subq_n3.key = srcpart.key);
 explain select * from masking_test_subq_n3 join srcpart on (masking_test_subq_n3.key = srcpart.key);
 select * from masking_test_subq_n3 join srcpart on (masking_test_subq_n3.key = srcpart.key);
 
+explain cbo select * from default.masking_test_subq_n3 where key > 0;
 explain select * from default.masking_test_subq_n3 where key > 0;
 select * from default.masking_test_subq_n3 where key > 0;
 
+explain cbo select * from masking_test_subq_n3 where masking_test_subq_n3.key > 0;
 explain select * from masking_test_subq_n3 where masking_test_subq_n3.key > 0;
 select * from masking_test_subq_n3 where masking_test_subq_n3.key > 0;
 
diff --git a/ql/src/test/queries/clientpositive/masking_4.q b/ql/src/test/queries/clientpositive/masking_4.q
index b4fc24a..726b706 100644
--- a/ql/src/test/queries/clientpositive/masking_4.q
+++ b/ql/src/test/queries/clientpositive/masking_4.q
@@ -26,7 +26,9 @@
 select * from masking_test_subq_n2;
 
 --should mask masking_test_subq_n2
-
+explain cbo
+with q1 as ( select * from masking_test_n11 where key = '5')
+select * from masking_test_subq_n2;
 explain
 with q1 as ( select * from masking_test_n11 where key = '5')
 select * from masking_test_subq_n2;
\ No newline at end of file
diff --git a/ql/src/test/queries/clientpositive/subquery_complex_correlation_predicates.q b/ql/src/test/queries/clientpositive/subquery_complex_correlation_predicates.q
new file mode 100644
index 0000000..a32eb91
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/subquery_complex_correlation_predicates.q
@@ -0,0 +1,78 @@
+-- Test cases with subqueries having complex correlation predicates. 
+
+-- HIVE-24957: Wrong results when subquery has COALESCE in correlation predicate
+create table author
+(
+    a_authorkey int,
+    a_name      varchar(50)
+);
+
+create table book
+(
+    b_bookkey   int,
+    b_title     varchar(50),
+    b_authorkey int
+);
+
+insert into author
+values (10, 'Victor Hugo');
+insert into author
+values (20, 'Alexandre Dumas');
+insert into author
+values (300, 'UNKNOWN1');
+insert into author
+values (null, 'UNKNOWN2');
+
+insert into book
+values (1, 'Les Miserables', 10);
+insert into book
+values (2, 'The Count of Monte Cristo', 20);
+insert into book
+values (3, 'Men Without Women', 30);
+insert into book
+values (4, 'Odyssey', null);
+
+explain cbo
+select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 300) = a.a_authorkey);
+
+select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 300) = a.a_authorkey);
+
+explain cbo
+select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400));
+
+select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400));
+
+explain cbo
+select b.b_title
+from book b
+where not exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400));
+
+select b.b_title
+from book b
+where not exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400));
\ No newline at end of file
diff --git a/ql/src/test/queries/clientpositive/subquery_in_invalid_intermediate_plan.q b/ql/src/test/queries/clientpositive/subquery_in_invalid_intermediate_plan.q
new file mode 100644
index 0000000..902cf25
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/subquery_in_invalid_intermediate_plan.q
@@ -0,0 +1,56 @@
+--! qt:dataset:part
+--! qt:dataset:src
+
+-- HIVE-24999: HiveSubQueryRemoveRule generates invalid plan for IN subquery with correlations
+-- Without the fix queries below fail with AssertionError at compilation time when -Dcalcite.debug is enabled. After
+-- Calcite 1.27.0 the queries would fail irrespective of the value of -Dcalcite.debug property.
+
+explain cbo
+select *
+from src b
+where b.key in (select a.key from src a where b.value = a.value and a.key > '9');
+
+explain cbo
+select count(*) as c
+from part as e
+where p_size + 100 IN (select p_partkey from part where p_name = e.p_name);
+
+explain cbo
+select *
+from part
+where p_name IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey);
+
+explain cbo
+select *
+from src b
+where b.key in (select key from src a where b.value > a.value and b.key < a.key);
+
+explain cbo select p_mfgr, p_name, p_size
+from part b where b.p_size in
+                  (select min(p_size)
+                   from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a
+                   where r <= 2 and b.p_mfgr = a.p_mfgr
+                  );
+
+select p_mfgr, p_name, p_size
+from part b where b.p_size in
+                  (select min(p_size)
+                   from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a
+                   where r <= 2 and b.p_mfgr = a.p_mfgr
+                  );
+
+explain cbo select *
+            from src b
+            where b.key in
+                  (select distinct a.key
+                   from src a
+                   where b.value <> a.key and a.key > '9'
+                  );
+
+select *
+from src b
+where b.key in
+      (select distinct a.key
+       from src a
+       where b.value <> a.key and a.key > '9'
+      );
\ No newline at end of file
diff --git a/ql/src/test/queries/clientpositive/subquery_notexists.q b/ql/src/test/queries/clientpositive/subquery_notexists.q
index 085ecd1..7413667 100644
--- a/ql/src/test/queries/clientpositive/subquery_notexists.q
+++ b/ql/src/test/queries/clientpositive/subquery_notexists.q
@@ -64,6 +64,8 @@
 ;
 
 --  bug in decorrelation where HiveProject gets multiple column with same name
+explain cbo 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);
 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);
 SELECT p1.p_name FROM part p1 LEFT JOIN (select p_type as p_col from part ) p2 WHERE NOT EXISTS
diff --git a/ql/src/test/queries/clientpositive/subquery_notin.q b/ql/src/test/queries/clientpositive/subquery_notin.q
index f863645..0bbaaa6 100644
--- a/ql/src/test/queries/clientpositive/subquery_notin.q
+++ b/ql/src/test/queries/clientpositive/subquery_notin.q
@@ -121,6 +121,7 @@
 select * from part where (p_partkey*p_size) NOT IN (select min(p_partkey) from part group by p_type);
 
 --lhs contains non-simple expression, corr
+explain cbo select count(*) as c from part as e where p_size + 100 NOT IN (select p_partkey from part where p_name = e.p_name);
 explain select count(*) as c from part as e where p_size + 100 NOT IN (select p_partkey from part where p_name = e.p_name);
 select count(*) as c from part as e where p_size + 100 NOT IN (select p_partkey from part where p_name = e.p_name);
 
@@ -128,6 +129,7 @@
 explain select * from part  where floor(p_retailprice) NOT IN (select floor(min(p_retailprice)) from part group by p_type);
 select * from part  where floor(p_retailprice) NOT IN (select floor(min(p_retailprice)) from part group by p_type);
 
+explain cbo select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey );
 explain select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey );
 select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey );
 
@@ -178,6 +180,7 @@
 SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0);
 
 -- corr
+explain cbo SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0 where t1_n0.c2=t2_n0.c1);
 explain SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0 where t1_n0.c2=t2_n0.c1);
 SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0 where t1_n0.c2=t2_n0.c1);
 
diff --git a/ql/src/test/results/clientpositive/llap/lineage3.q.out b/ql/src/test/results/clientpositive/llap/lineage3.q.out
index 83233df..45b1b8f 100644
--- a/ql/src/test/results/clientpositive/llap/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/llap/lineage3.q.out
@@ -184,7 +184,7 @@
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-{"version":"1.0","engine":"tez","database":"default","hash":"94e9cc0a67801fe1503a3cb0c5029d59","queryText":"select * from src1 a\nwhere exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > 300.0D)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(a.key = a.key)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"UDFToDouble((UDFToInteger(b.ctinyint) + 300)) is not null","edgeType":"PREDICATE"},{"sources":[4,2],"targets":[0,1],"expression":"(UDFToDouble((UDFToInteger(b.ctinyint) + 300)) = UDFToDouble(a.key))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
+{"version":"1.0","engine":"tez","database":"default","hash":"94e9cc0a67801fe1503a3cb0c5029d59","queryText":"select * from src1 a\nwhere exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > 300.0D)","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"UDFToDouble((UDFToInteger(b.ctinyint) + 300)) is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 311	val_311
 Warning: Shuffle Join MERGEJOIN[37][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select key, value from src1
@@ -202,7 +202,7 @@
 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],"expression":"(a.key = a.key)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"UDFToDouble((UDFToInteger(b.ctinyint) + 300)) is not null","edgeType":"PREDICATE"},{"sources":[4,2],"targets":[0,1],"expression":"(UDFToDouble((UDFToInteger(b.ctinyint) + 300)) = UDFToDouble(a.key))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
+{"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,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"UDFToDouble((UDFToInteger(b.ctinyint) + 300)) is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 369	
 401	val_401
 406	val_406
@@ -410,7 +410,7 @@
 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],"expression":"(a.key = a.key)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"UDFToDouble((UDFToInteger(b.ctinyint) + 300)) is not null","edgeType":"PREDICATE"},{"sources":[4,2],"targets":[0,1],"expression":"(UDFToDouble((UDFToInteger(b.ctinyint) + 300)) = UDFToDouble(a.key))","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"true is null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
+{"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,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"UDFToDouble((UDFToInteger(b.ctinyint) + 300)) is not null","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"true is null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 369	
 401	val_401
 406	val_406
diff --git a/ql/src/test/results/clientpositive/llap/masking_12.q.out b/ql/src/test/results/clientpositive/llap/masking_12.q.out
index 08c8916..45dcd41 100644
--- a/ql/src/test/results/clientpositive/llap/masking_12.q.out
+++ b/ql/src/test/results/clientpositive/llap/masking_12.q.out
@@ -94,6 +94,33 @@
 POSTHOOK: Output: default@v1_n9
 POSTHOOK: Lineage: v1_n9.key SIMPLE [(masking_test_subq_n1)masking_test_subq_n1.FieldSchema(name:key, type:int, comment:null), ]
 POSTHOOK: Lineage: v1_n9.value SIMPLE [(masking_test_subq_n1)masking_test_subq_n1.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: explain cbo
+select * from `v1_n9`
+limit 20
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n1
+PREHOOK: Input: default@src
+PREHOOK: Input: default@v1_n9
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+select * from `v1_n9`
+limit 20
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n1
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@v1_n9
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortLimit(fetch=[20])
+  HiveProject(key=[$0], value=[$1])
+    HiveSemiJoin(condition=[AND(=($0, $3), =($4, $2))], joinType=[semi])
+      HiveProject(key=[$0], value=[$1], key1=[CAST($0):DOUBLE])
+        HiveFilter(condition=[IS NOT NULL(CAST($0):DOUBLE)])
+          HiveTableScan(table=[[default, masking_test_subq_n1]], table:alias=[masking_test_subq_n1])
+      HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+        HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+          HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain
 select * from `v1_n9`
 limit 20
@@ -119,69 +146,60 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
-        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: masking_test_subq_n1
-                  filterExpr: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                  filterExpr: UDFToDouble(key) is not null (type: boolean)
                   properties:
                     insideView TRUE
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                    predicate: UDFToDouble(key) is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col2 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col2 (type: double)
+                        Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   properties:
                     insideView TRUE
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -191,8 +209,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col2 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                 Limit
@@ -205,52 +223,6 @@
                         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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/masking_3.q.out b/ql/src/test/results/clientpositive/llap/masking_3.q.out
index 2a73f5d..d8df9d6 100644
--- a/ql/src/test/results/clientpositive/llap/masking_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/masking_3.q.out
@@ -10,6 +10,26 @@
 POSTHOOK: Output: default@masking_test_subq_n3
 POSTHOOK: Lineage: masking_test_subq_n3.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: masking_test_subq_n3.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: explain cbo select * from masking_test_subq_n3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n3
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from masking_test_subq_n3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n3
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(key=[$0], value=[$1])
+  HiveSemiJoin(condition=[AND(=($0, $3), =($4, $2))], joinType=[semi])
+    HiveProject(key=[$0], value=[$1], key1=[CAST($0):DOUBLE])
+      HiveFilter(condition=[IS NOT NULL(CAST($0):DOUBLE)])
+        HiveTableScan(table=[[default, masking_test_subq_n3]], table:alias=[masking_test_subq_n3])
+    HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain select * from masking_test_subq_n3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@masking_test_subq_n3
@@ -29,65 +49,56 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
-        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: masking_test_subq_n3
-                  filterExpr: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                  filterExpr: UDFToDouble(key) is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                    predicate: UDFToDouble(key) is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col2 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col2 (type: double)
+                        Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -97,8 +108,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col2 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
@@ -108,52 +119,6 @@
                       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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -671,6 +636,26 @@
 97	val_97
 98	val_98
 98	val_98
+PREHOOK: query: explain cbo select * from masking_test_subq_n3 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n3
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from masking_test_subq_n3 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n3
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(key=[$0], value=[$1])
+  HiveSemiJoin(condition=[AND(=($0, $3), =($4, $2))], joinType=[semi])
+    HiveProject(key=[$0], value=[$1], key1=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(>($0, 0), IS NOT NULL(CAST($0):DOUBLE))])
+        HiveTableScan(table=[[default, masking_test_subq_n3]], table:alias=[masking_test_subq_n3])
+    HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain select * from masking_test_subq_n3 where key > 0
 PREHOOK: type: QUERY
 PREHOOK: Input: default@masking_test_subq_n3
@@ -690,9 +675,7 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
-        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -705,50 +688,43 @@
                     predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col2 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col2 (type: double)
+                        Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -758,8 +734,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col2 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
@@ -769,52 +745,6 @@
                       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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -1329,6 +1259,26 @@
 97	val_97
 98	val_98
 98	val_98
+PREHOOK: query: explain cbo select key from masking_test_subq_n3 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n3
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select key from masking_test_subq_n3 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n3
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(key=[$0])
+  HiveSemiJoin(condition=[AND(=($0, $2), =($3, $1))], joinType=[semi])
+    HiveProject(key=[$0], key1=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(>($0, 0), IS NOT NULL(CAST($0):DOUBLE))])
+        HiveTableScan(table=[[default, masking_test_subq_n3]], table:alias=[masking_test_subq_n3])
+    HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain select key from masking_test_subq_n3 where key > 0
 PREHOOK: type: QUERY
 PREHOOK: Input: default@masking_test_subq_n3
@@ -1348,9 +1298,7 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
-        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -1363,49 +1311,43 @@
                     predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 500 Data size: 6000 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                        Statistics: Num rows: 500 Data size: 6000 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -1415,8 +1357,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col1 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
@@ -1426,52 +1368,6 @@
                       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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -1986,6 +1882,26 @@
 97
 98
 98
+PREHOOK: query: explain cbo select value from masking_test_subq_n3 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n3
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select value from masking_test_subq_n3 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n3
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(value=[$1])
+  HiveSemiJoin(condition=[AND(=($0, $3), =($4, $2))], joinType=[semi])
+    HiveProject(key=[$0], value=[$1], key1=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(>($0, 0), IS NOT NULL(CAST($0):DOUBLE))])
+        HiveTableScan(table=[[default, masking_test_subq_n3]], table:alias=[masking_test_subq_n3])
+    HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain select value from masking_test_subq_n3 where key > 0
 PREHOOK: type: QUERY
 PREHOOK: Input: default@masking_test_subq_n3
@@ -2005,9 +1921,7 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
-        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2020,50 +1934,43 @@
                     predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col2 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col2 (type: double)
+                        Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: string)
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -2073,8 +1980,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col2 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col1
                 Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
@@ -2088,52 +1995,6 @@
                         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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -2648,6 +2509,41 @@
 val_97
 val_98
 val_98
+PREHOOK: query: explain cbo select * from masking_test_subq_n3 join srcpart on (masking_test_subq_n3.key = srcpart.key)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n3
+PREHOOK: Input: default@src
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from masking_test_subq_n3 join srcpart on (masking_test_subq_n3.key = srcpart.key)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n3
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(key=[$5], value=[$6], key1=[$0], value1=[$1], ds=[$2], hr=[$3])
+  HiveJoin(condition=[=($7, $4)], joinType=[inner], algorithm=[none], cost=[not available])
+    HiveProject(key=[$0], value=[$1], ds=[$2], hr=[$3], CAST=[CAST($0):DOUBLE])
+      HiveFilter(condition=[IS NOT NULL(CAST($0):DOUBLE)])
+        HiveTableScan(table=[[default, srcpart]], table:alias=[srcpart])
+    HiveProject(key=[$0], value=[$1], CAST=[CAST($0):DOUBLE])
+      HiveSemiJoin(condition=[AND(=($0, $3), =($4, $2))], joinType=[semi])
+        HiveProject(key=[$0], value=[$1], key1=[CAST($0):DOUBLE])
+          HiveFilter(condition=[IS NOT NULL(CAST($0):DOUBLE)])
+            HiveTableScan(table=[[default, masking_test_subq_n3]], table:alias=[masking_test_subq_n3])
+        HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+          HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+            HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain select * from masking_test_subq_n3 join srcpart on (masking_test_subq_n3.key = srcpart.key)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@masking_test_subq_n3
@@ -2678,9 +2574,7 @@
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 5 <- Map 3 (SIMPLE_EDGE)
-        Reducer 6 <- Map 7 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2709,56 +2603,49 @@
             Map Operator Tree:
                 TableScan
                   alias: masking_test_subq_n3
-                  filterExpr: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                  filterExpr: UDFToDouble(key) is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                    predicate: UDFToDouble(key) is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col2 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col2 (type: double)
+                        Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 7 
+        Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -2790,8 +2677,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col2 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
@@ -2805,52 +2692,6 @@
                     Map-reduce partition columns: _col2 (type: double)
                     Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: string)
-        Reducer 5 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 6 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -6990,6 +6831,26 @@
 98	val_98	98	val_98	2008-04-09	12
 98	val_98	98	val_98	2008-04-09	12
 98	val_98	98	val_98	2008-04-09	12
+PREHOOK: query: explain cbo select * from default.masking_test_subq_n3 where key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n3
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from default.masking_test_subq_n3 where key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n3
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(key=[$0], value=[$1])
+  HiveSemiJoin(condition=[AND(=($0, $3), =($4, $2))], joinType=[semi])
+    HiveProject(key=[$0], value=[$1], key1=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(>($0, 0), IS NOT NULL(CAST($0):DOUBLE))])
+        HiveTableScan(table=[[default, masking_test_subq_n3]], table:alias=[masking_test_subq_n3])
+    HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain select * from default.masking_test_subq_n3 where key > 0
 PREHOOK: type: QUERY
 PREHOOK: Input: default@masking_test_subq_n3
@@ -7009,9 +6870,7 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
-        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -7024,50 +6883,43 @@
                     predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col2 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col2 (type: double)
+                        Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -7077,8 +6929,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col2 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
@@ -7088,52 +6940,6 @@
                       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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -7648,6 +7454,26 @@
 97	val_97
 98	val_98
 98	val_98
+PREHOOK: query: explain cbo select * from masking_test_subq_n3 where masking_test_subq_n3.key > 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n3
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from masking_test_subq_n3 where masking_test_subq_n3.key > 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n3
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(key=[$0], value=[$1])
+  HiveSemiJoin(condition=[AND(=($0, $3), =($4, $2))], joinType=[semi])
+    HiveProject(key=[$0], value=[$1], key1=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(>($0, 0), IS NOT NULL(CAST($0):DOUBLE))])
+        HiveTableScan(table=[[default, masking_test_subq_n3]], table:alias=[masking_test_subq_n3])
+    HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain select * from masking_test_subq_n3 where masking_test_subq_n3.key > 0
 PREHOOK: type: QUERY
 PREHOOK: Input: default@masking_test_subq_n3
@@ -7667,9 +7493,7 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
-        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -7682,50 +7506,43 @@
                     predicate: ((key > 0) and UDFToDouble(key) is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col2 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col2 (type: double)
+                        Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -7735,8 +7552,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col2 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
@@ -7746,52 +7563,6 @@
                       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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/masking_4.q.out b/ql/src/test/results/clientpositive/llap/masking_4.q.out
index 63c31c4..f3eee2c 100644
--- a/ql/src/test/results/clientpositive/llap/masking_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/masking_4.q.out
@@ -114,6 +114,30 @@
               outputColumnNames: _col0, _col1
               ListSink
 
+PREHOOK: query: explain cbo
+with q1 as ( select * from masking_test_n11 where key = '5')
+select * from masking_test_subq_n2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@masking_test_subq_n2
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+with q1 as ( select * from masking_test_n11 where key = '5')
+select * from masking_test_subq_n2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@masking_test_subq_n2
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(key=[$0], value=[$1])
+  HiveSemiJoin(condition=[AND(=($0, $3), =($4, $2))], joinType=[semi])
+    HiveProject(key=[$0], value=[$1], key1=[CAST($0):DOUBLE])
+      HiveFilter(condition=[IS NOT NULL(CAST($0):DOUBLE)])
+        HiveTableScan(table=[[default, masking_test_subq_n2]], table:alias=[masking_test_subq_n2])
+    HiveProject(key=[$0], key0=[CAST($0):DOUBLE])
+      HiveFilter(condition=[AND(IS NOT NULL(CAST($0):DOUBLE), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, src]], table:alias=[src])
+
 PREHOOK: query: explain
 with q1 as ( select * from masking_test_n11 where key = '5')
 select * from masking_test_subq_n2
@@ -137,65 +161,56 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE)
-        Reducer 4 <- Map 5 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: masking_test_subq_n2
-                  filterExpr: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                  filterExpr: UDFToDouble(key) is not null (type: boolean)
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (key is not null and UDFToDouble(key) is not null) (type: boolean)
+                    predicate: UDFToDouble(key) is not null (type: boolean)
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: key (type: int), value (type: string)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: key (type: int), value (type: string), UDFToDouble(key) (type: double)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: UDFToDouble(_col0) (type: double), _col0 (type: int)
+                        key expressions: UDFToDouble(_col0) (type: double), _col2 (type: double)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col0 (type: int)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col1 (type: string)
-                    Group By Operator
-                      keys: key (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 303 Data size: 1212 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: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
+                        Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col2 (type: double)
+                        Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  filterExpr: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: UDFToDouble(key) is not null (type: boolean)
+                    predicate: (UDFToDouble(key) is not null and key is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), UDFToDouble(key) (type: double)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
+                      Group By Operator
+                        keys: _col0 (type: string), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: double)
+                          Statistics: Num rows: 316 Data size: 30020 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -205,8 +220,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 UDFToDouble(_col0) (type: double), _col0 (type: int)
-                  1 UDFToDouble(_col0) (type: double), _col1 (type: int)
+                  0 UDFToDouble(_col0) (type: double), _col2 (type: double)
+                  1 UDFToDouble(_col0) (type: double), _col1 (type: double)
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
@@ -216,52 +231,6 @@
                       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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 303 Data size: 1212 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 303 Data size: 3636 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int)
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 479 Data size: 43589 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: UDFToDouble(_col0) (type: double), _col1 (type: int)
-                      Statistics: Num rows: 303 Data size: 27573 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/subquery_ANY.q.out b/ql/src/test/results/clientpositive/llap/subquery_ANY.q.out
index 0c3ac6c..c1d56bd 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_ANY.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_ANY.q.out
@@ -1471,10 +1471,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part_null_n1
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_type is not null) (type: boolean)
                   Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (p_type is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_type is not null) (type: boolean)
                     Statistics: Num rows: 25 Data size: 15475 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)
@@ -1671,10 +1671,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part_null_n1
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_type is not null) (type: boolean)
                   Statistics: Num rows: 27 Data size: 16713 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (p_type is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_type is not null) (type: boolean)
                     Statistics: Num rows: 25 Data size: 15475 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)
diff --git a/ql/src/test/results/clientpositive/llap/subquery_complex_correlation_predicates.q.out b/ql/src/test/results/clientpositive/llap/subquery_complex_correlation_predicates.q.out
new file mode 100644
index 0000000..fa9768c
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/subquery_complex_correlation_predicates.q.out
@@ -0,0 +1,292 @@
+PREHOOK: query: create table author
+(
+    a_authorkey int,
+    a_name      varchar(50)
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@author
+POSTHOOK: query: create table author
+(
+    a_authorkey int,
+    a_name      varchar(50)
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@author
+PREHOOK: query: create table book
+(
+    b_bookkey   int,
+    b_title     varchar(50),
+    b_authorkey int
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@book
+POSTHOOK: query: create table book
+(
+    b_bookkey   int,
+    b_title     varchar(50),
+    b_authorkey int
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@book
+PREHOOK: query: insert into author
+values (10, 'Victor Hugo')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@author
+POSTHOOK: query: insert into author
+values (10, 'Victor Hugo')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@author
+POSTHOOK: Lineage: author.a_authorkey SCRIPT []
+POSTHOOK: Lineage: author.a_name SCRIPT []
+PREHOOK: query: insert into author
+values (20, 'Alexandre Dumas')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@author
+POSTHOOK: query: insert into author
+values (20, 'Alexandre Dumas')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@author
+POSTHOOK: Lineage: author.a_authorkey SCRIPT []
+POSTHOOK: Lineage: author.a_name SCRIPT []
+PREHOOK: query: insert into author
+values (300, 'UNKNOWN1')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@author
+POSTHOOK: query: insert into author
+values (300, 'UNKNOWN1')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@author
+POSTHOOK: Lineage: author.a_authorkey SCRIPT []
+POSTHOOK: Lineage: author.a_name SCRIPT []
+PREHOOK: query: insert into author
+values (null, 'UNKNOWN2')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@author
+POSTHOOK: query: insert into author
+values (null, 'UNKNOWN2')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@author
+POSTHOOK: Lineage: author.a_authorkey EXPRESSION []
+POSTHOOK: Lineage: author.a_name SCRIPT []
+PREHOOK: query: insert into book
+values (1, 'Les Miserables', 10)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@book
+POSTHOOK: query: insert into book
+values (1, 'Les Miserables', 10)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@book
+POSTHOOK: Lineage: book.b_authorkey SCRIPT []
+POSTHOOK: Lineage: book.b_bookkey SCRIPT []
+POSTHOOK: Lineage: book.b_title SCRIPT []
+PREHOOK: query: insert into book
+values (2, 'The Count of Monte Cristo', 20)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@book
+POSTHOOK: query: insert into book
+values (2, 'The Count of Monte Cristo', 20)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@book
+POSTHOOK: Lineage: book.b_authorkey SCRIPT []
+POSTHOOK: Lineage: book.b_bookkey SCRIPT []
+POSTHOOK: Lineage: book.b_title SCRIPT []
+PREHOOK: query: insert into book
+values (3, 'Men Without Women', 30)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@book
+POSTHOOK: query: insert into book
+values (3, 'Men Without Women', 30)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@book
+POSTHOOK: Lineage: book.b_authorkey SCRIPT []
+POSTHOOK: Lineage: book.b_bookkey SCRIPT []
+POSTHOOK: Lineage: book.b_title SCRIPT []
+PREHOOK: query: insert into book
+values (4, 'Odyssey', null)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@book
+POSTHOOK: query: insert into book
+values (4, 'Odyssey', null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@book
+POSTHOOK: Lineage: book.b_authorkey EXPRESSION []
+POSTHOOK: Lineage: book.b_bookkey SCRIPT []
+POSTHOOK: Lineage: book.b_title SCRIPT []
+PREHOOK: query: explain cbo
+select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 300) = a.a_authorkey)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@author
+PREHOOK: Input: default@book
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 300) = a.a_authorkey)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@author
+POSTHOOK: Input: default@book
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(b_title=[$0])
+  HiveSemiJoin(condition=[=($1, $2)], joinType=[semi])
+    HiveProject(b_title=[$1], $f7=[CASE(IS NOT NULL($2), $2, 300)])
+      HiveTableScan(table=[[default, book]], table:alias=[b])
+    HiveProject(a_authorkey=[$0])
+      HiveFilter(condition=[IS NOT NULL($0)])
+        HiveTableScan(table=[[default, author]], table:alias=[a])
+
+PREHOOK: query: select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 300) = a.a_authorkey)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@author
+PREHOOK: Input: default@book
+#### A masked pattern was here ####
+POSTHOOK: query: select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 300) = a.a_authorkey)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@author
+POSTHOOK: Input: default@book
+#### A masked pattern was here ####
+Les Miserables
+The Count of Monte Cristo
+Odyssey
+PREHOOK: query: explain cbo
+select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@author
+PREHOOK: Input: default@book
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@author
+POSTHOOK: Input: default@book
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(b_title=[$0])
+  HiveSemiJoin(condition=[=($1, $2)], joinType=[semi])
+    HiveProject(b_title=[$1], $f7=[CASE(IS NOT NULL($2), $2, 400)])
+      HiveTableScan(table=[[default, book]], table:alias=[b])
+    HiveProject($f6=[CASE(IS NOT NULL($0), $0, 400)])
+      HiveTableScan(table=[[default, author]], table:alias=[a])
+
+PREHOOK: query: select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@author
+PREHOOK: Input: default@book
+#### A masked pattern was here ####
+POSTHOOK: query: select b.b_title
+from book b
+where exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@author
+POSTHOOK: Input: default@book
+#### A masked pattern was here ####
+Les Miserables
+The Count of Monte Cristo
+Odyssey
+PREHOOK: query: explain cbo
+select b.b_title
+from book b
+where not exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@author
+PREHOOK: Input: default@book
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+select b.b_title
+from book b
+where not exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@author
+POSTHOOK: Input: default@book
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(b_title=[$0])
+  HiveAntiJoin(condition=[=($1, $3)], joinType=[anti])
+    HiveProject(b_title=[$1], $f7=[CASE(IS NOT NULL($2), $2, 400)])
+      HiveTableScan(table=[[default, book]], table:alias=[b])
+    HiveProject(literalTrue=[true], $f6=[CASE(IS NOT NULL($0), $0, 400)])
+      HiveTableScan(table=[[default, author]], table:alias=[a])
+
+PREHOOK: query: select b.b_title
+from book b
+where not exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@author
+PREHOOK: Input: default@book
+#### A masked pattern was here ####
+POSTHOOK: query: select b.b_title
+from book b
+where not exists
+          (select a_authorkey
+           from author a
+           where coalesce(b.b_authorkey, 400) = coalesce(a.a_authorkey, 400))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@author
+POSTHOOK: Input: default@book
+#### A masked pattern was here ####
+Men Without Women
diff --git a/ql/src/test/results/clientpositive/llap/subquery_corr.q.out b/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
index a8be7cc..7583e62 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_corr.q.out
@@ -766,22 +766,22 @@
                     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), key (type: string), value (type: string)
-                      outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 500 Data size: 132500 Basic stats: COMPLETE Column stats: COMPLETE
+                      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), _col2 (type: string)
+                        keys: _col0 (type: string), _col1 (type: string)
                         minReductionHashAggr: 0.4
                         mode: hash
-                        outputColumnNames: _col0, _col1, _col2
-                        Statistics: Num rows: 500 Data size: 132500 Basic stats: COMPLETE Column stats: COMPLETE
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 316 Data size: 56248 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: 132500 Basic stats: COMPLETE Column stats: COMPLETE
-                          value expressions: _col1 (type: string), _col2 (type: string)
+                          Statistics: Num rows: 316 Data size: 56248 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col1 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -793,16 +793,16 @@
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
-                outputColumnNames: _col0, _col1, _col3, _col4
-                residual filter predicates: {(_col0 < _col3)} {(_col1 > _col4)}
-                Statistics: Num rows: 55 Data size: 19580 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col1, _col3
+                residual filter predicates: {(_col1 > _col3)}
+                Statistics: Num rows: 166 Data size: 44654 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in.q.out b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
index cf011ce..930d2c2 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in.q.out
@@ -1878,14 +1878,14 @@
                     predicate: (p_name is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: p_name (type: string), p_size (type: int)
+                      expressions: p_name (type: string), (p_size + 100) (type: int)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: string), (_col1 + 100) (type: int)
+                        key expressions: _col0 (type: string), _col1 (type: int)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: string), (_col1 + 100) (type: int)
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                         Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -1923,7 +1923,7 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 _col0 (type: string), (_col1 + 100) (type: int)
+                  0 _col0 (type: string), _col1 (type: int)
                   1 _col0 (type: string), _col1 (type: int)
                 Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
@@ -2142,21 +2142,21 @@
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_size is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_size is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (p_size is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_size 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
+                      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), (p_size + 121150) (type: int)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+                      Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string), _col5 (type: int)
-                        null sort order: zz
-                        sort order: ++
-                        Map-reduce partition columns: _col1 (type: string), _col5 (type: int)
-                        Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                        key expressions: _col1 (type: string), _col5 (type: int), _col9 (type: int)
+                        null sort order: zzz
+                        sort order: +++
+                        Map-reduce partition columns: _col1 (type: string), _col5 (type: int), _col9 (type: int)
+                        Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
@@ -2164,27 +2164,27 @@
             Map Operator Tree:
                 TableScan
                   alias: p
-                  filterExpr: (((p_size + 121150) = p_partkey) and p_size is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_size is not null and p_partkey is not null and p_name is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (((p_size + 121150) = p_partkey) and p_size is not null and p_name is not null) (type: boolean)
-                    Statistics: Num rows: 13 Data size: 1677 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (p_size is not null and p_partkey is not null and p_name is not null) (type: boolean)
+                    Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: p_name (type: string), p_size (type: int)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: p_name (type: string), p_size (type: int), p_partkey (type: int)
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        keys: _col0 (type: string), _col1 (type: int)
+                        keys: _col0 (type: string), _col1 (type: int), _col2 (type: int)
                         minReductionHashAggr: 0.4
                         mode: hash
-                        outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
+                        outputColumnNames: _col0, _col1, _col2
+                        Statistics: Num rows: 25 Data size: 3225 Basic stats: COMPLETE Column stats: COMPLETE
                         Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: int)
-                          null sort order: zz
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
-                          Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
+                          key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int)
+                          null sort order: zzz
+                          sort order: +++
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: int)
+                          Statistics: Num rows: 25 Data size: 3225 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -2194,13 +2194,13 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 _col1 (type: string), _col5 (type: int)
-                  1 _col0 (type: string), _col1 (type: int)
+                  0 _col1 (type: string), _col5 (type: int), _col9 (type: int)
+                  1 _col0 (type: string), _col1 (type: int), _col2 (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
-                Statistics: Num rows: 13 Data size: 8047 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 13 Data size: 8047 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -2246,10 +2246,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_partkey is not null and p_size is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_partkey is not null and p_name is not null and p_size is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (p_partkey is not null and p_size is not null and p_name is not null) (type: boolean)
+                    predicate: (p_partkey is not null and p_name is not null and p_size 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)
@@ -2374,10 +2374,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (p_brand is not null and p_type is not null) (type: boolean)
+                  filterExpr: (p_type is not null and p_brand is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 8242 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (p_brand is not null and p_type is not null) (type: boolean)
+                    predicate: (p_type is not null and p_brand is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 8242 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_name (type: string), p_type (type: string), p_brand (type: string)
@@ -2495,19 +2495,17 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: ((p_size is not null and p_type is not null) or p_size is not null) (type: boolean)
+                  filterExpr: (p_type is not null and p_size is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (p_size is not null and p_type is not null) (type: boolean)
+                    predicate: (p_type is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_name (type: string), p_type (type: string), (p_size + 1) (type: int)
@@ -2520,36 +2518,33 @@
                         Map-reduce partition columns: _col1 (type: string), _col2 (type: int)
                         Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: part
+                  filterExpr: (p_size is not null and p_type is not null) (type: boolean)
+                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (p_size is not null and p_type is not null) (type: boolean)
+                    Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_type (type: string), (p_size + 1) (type: int)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: int)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: int)
-                        Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
-                  Filter Operator
-                    predicate: p_size is not null (type: boolean)
-                    Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: (p_size + 1) (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        keys: _col0 (type: int)
+                        keys: _col0 (type: string), _col1 (type: int)
                         minReductionHashAggr: 0.4
                         mode: hash
-                        outputColumnNames: _col0
-                        Statistics: Num rows: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 24 Data size: 2592 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: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
+                          key expressions: _col0 (type: string), _col1 (type: int)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
+                          Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -2570,47 +2565,6 @@
                       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: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), _col2 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string), _col1 (type: int)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      key expressions: _col0 (type: string), _col1 (type: int)
-                      null sort order: zz
-                      sort order: ++
-                      Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
-                      Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 4 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 21 Data size: 84 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: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -3578,10 +3532,10 @@
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: (p_mfgr is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_mfgr is not null) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (p_mfgr is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_mfgr is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 5798 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_name (type: string), p_mfgr (type: string), p_size (type: int)
@@ -3708,7 +3662,7 @@
             Map Operator Tree:
                 TableScan
                   alias: p
-                  filterExpr: ((p_type is not null and p_name is not null) or (p_size is not null and p_name is not null)) (type: boolean)
+                  filterExpr: ((p_type is not null and p_name is not null) or (p_name is not null and p_size is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_name is not null) (type: boolean)
@@ -3725,7 +3679,7 @@
                         Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
                   Filter Operator
-                    predicate: (p_size is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_partkey (type: int), p_name (type: string), p_size (type: int)
@@ -3875,7 +3829,7 @@
             Map Operator Tree:
                 TableScan
                   alias: p
-                  filterExpr: ((p_size is not null and p_type is not null and p_name is not null) or (p_size is not null and p_name is not null)) (type: boolean)
+                  filterExpr: ((p_size is not null and p_type is not null and p_name is not null) or (p_name is not null and p_size is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_size is not null and p_type is not null and p_name is not null) (type: boolean)
@@ -3892,7 +3846,7 @@
                         Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
                   Filter Operator
-                    predicate: (p_size is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_partkey (type: int), p_name (type: string), p_size (type: int)
@@ -4044,10 +3998,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: ((p_type is not null and p_size is not null and p_name is not null) or (p_type is not null and p_name is not null)) (type: boolean)
+                  filterExpr: ((p_name is not null and p_type is not null and p_size is not null) or (p_type is not null and p_name is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 6058 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (p_type is not null and p_size is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_type is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 6058 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_partkey (type: int), p_name (type: string), p_type (type: string), p_size (type: int)
@@ -4177,7 +4131,7 @@
             Map Operator Tree:
                 TableScan
                   alias: p
-                  filterExpr: ((p_type is not null and p_name is not null) or (p_size is not null and p_name is not null)) (type: boolean)
+                  filterExpr: ((p_type is not null and p_name is not null) or (p_name is not null and p_size is not null)) (type: boolean)
                   Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (p_type is not null and p_name is not null) (type: boolean)
@@ -4194,7 +4148,7 @@
                         Statistics: Num rows: 26 Data size: 5850 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
                   Filter Operator
-                    predicate: (p_size is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_size is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: p_partkey (type: int), p_name (type: string), p_size (type: int)
@@ -5585,10 +5539,10 @@
             Map Operator Tree:
                 TableScan
                   alias: t_n22
-                  filterExpr: (j is not null and UDFToLong(i) is not null) (type: boolean)
+                  filterExpr: (UDFToLong(i) is not null and j is not null) (type: boolean)
                   Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (j is not null and UDFToLong(i) is not null) (type: boolean)
+                    predicate: (UDFToLong(i) is not null and j is not null) (type: boolean)
                     Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: i (type: int), j (type: int)
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 c10bc05..81dbd62 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
@@ -1009,10 +1009,10 @@
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: ((value is not null and key is not null) or (key > '9')) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or (key > '9')) (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)
+                    predicate: (key is not null and value is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in_invalid_intermediate_plan.q.out b/ql/src/test/results/clientpositive/llap/subquery_in_invalid_intermediate_plan.q.out
new file mode 100644
index 0000000..c9803e4
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/subquery_in_invalid_intermediate_plan.q.out
@@ -0,0 +1,211 @@
+PREHOOK: query: explain cbo
+select *
+from src b
+where b.key in (select a.key from src a where b.value = a.value and a.key > '9')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+select *
+from src b
+where b.key in (select a.key from src a where b.value = a.value and a.key > '9')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSemiJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[semi])
+  HiveProject(key=[$0], value=[$1])
+    HiveFilter(condition=[AND(>($0, _UTF-16LE'9'), IS NOT NULL($1))])
+      HiveTableScan(table=[[default, src]], table:alias=[b])
+  HiveProject(key=[$0], value=[$1])
+    HiveFilter(condition=[AND(>($0, _UTF-16LE'9'), IS NOT NULL($1))])
+      HiveTableScan(table=[[default, src]], table:alias=[a])
+
+PREHOOK: query: explain cbo
+select count(*) as c
+from part as e
+where p_size + 100 IN (select p_partkey from part where p_name = e.p_name)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+select count(*) as c
+from part as e
+where p_size + 100 IN (select p_partkey from part where p_name = e.p_name)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveSemiJoin(condition=[AND(=($3, $0), =($1, $2))], joinType=[semi])
+    HiveProject(p_name=[$1], $f13=[+($5, 100)])
+      HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($5))])
+        HiveTableScan(table=[[default, part]], table:alias=[e])
+    HiveProject(p_partkey=[$0], p_name=[$1])
+      HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($0))])
+        HiveTableScan(table=[[default, part]], table:alias=[part])
+
+PREHOOK: query: explain cbo
+select *
+from part
+where p_name IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+select *
+from part
+where p_name IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(p_partkey=[$0], p_name=[$1], p_mfgr=[$2], p_brand=[$3], p_type=[$4], p_size=[$5], p_container=[$6], p_retailprice=[$7], p_comment=[$8])
+  HiveSemiJoin(condition=[AND(=($1, $10), =($11, $5), =($9, $12))], joinType=[semi])
+    HiveProject(p_partkey=[$0], p_name=[$1], p_mfgr=[$2], p_brand=[$3], p_type=[$4], p_size=[$5], p_container=[$6], p_retailprice=[$7], p_comment=[$8], $f15=[+($5, 121150)])
+      HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($5))])
+        HiveTableScan(table=[[default, part]], table:alias=[part])
+    HiveProject(p_name=[$1], p_size=[$5], p_partkey=[$0])
+      HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($0), IS NOT NULL($1))])
+        HiveTableScan(table=[[default, part]], table:alias=[p])
+
+PREHOOK: query: explain cbo
+select *
+from src b
+where b.key in (select key from src a where b.value > a.value and b.key < a.key)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo
+select *
+from src b
+where b.key in (select key from src a where b.value > a.value and b.key < a.key)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSemiJoin(condition=[AND(=($2, $0), >($1, $3))], joinType=[semi])
+  HiveProject(key=[$0], value=[$1])
+    HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($1))])
+      HiveTableScan(table=[[default, src]], table:alias=[b])
+  HiveProject(key=[$0], value=[$1])
+    HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($0))])
+      HiveTableScan(table=[[default, src]], table:alias=[a])
+
+PREHOOK: query: explain cbo select p_mfgr, p_name, p_size
+from part b where b.p_size in
+                  (select min(p_size)
+                   from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a
+                   where r <= 2 and b.p_mfgr = a.p_mfgr
+                  )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select p_mfgr, p_name, p_size
+from part b where b.p_size in
+                  (select min(p_size)
+                   from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a
+                   where r <= 2 and b.p_mfgr = a.p_mfgr
+                  )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(p_mfgr=[$1], p_name=[$0], p_size=[$2])
+  HiveSemiJoin(condition=[AND(=($1, $4), =($2, $3))], joinType=[semi])
+    HiveProject(p_name=[$1], p_mfgr=[$2], p_size=[$5])
+      HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($5))])
+        HiveTableScan(table=[[default, part]], table:alias=[b])
+    HiveProject(_o__c0=[$1], p_mfgr=[$0])
+      HiveFilter(condition=[IS NOT NULL($1)])
+        HiveAggregate(group=[{0}], agg#0=[min($1)])
+          HiveFilter(condition=[<=($2, 2)])
+            HiveProject(p_mfgr=[$2], p_size=[$5], rank_window_0=[rank() OVER (PARTITION BY $2 ORDER BY $5 NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)])
+              HiveFilter(condition=[IS NOT NULL($2)])
+                HiveTableScan(table=[[default, part]], table:alias=[part])
+
+PREHOOK: query: select p_mfgr, p_name, p_size
+from part b where b.p_size in
+                  (select min(p_size)
+                   from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a
+                   where r <= 2 and b.p_mfgr = a.p_mfgr
+                  )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: select p_mfgr, p_name, p_size
+from part b where b.p_size in
+                  (select min(p_size)
+                   from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a
+                   where r <= 2 and b.p_mfgr = a.p_mfgr
+                  )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+Manufacturer#1	almond antique burnished rose metallic	2
+Manufacturer#1	almond antique burnished rose metallic	2
+Manufacturer#2	almond aquamarine midnight light salmon	2
+Manufacturer#3	almond antique misty red olive	1
+Manufacturer#4	almond aquamarine yellow dodger mint	7
+Manufacturer#5	almond antique sky peru orange	2
+PREHOOK: query: explain cbo select *
+            from src b
+            where b.key in
+                  (select distinct a.key
+                   from src a
+                   where b.value <> a.key and a.key > '9'
+                  )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select *
+            from src b
+            where b.key in
+                  (select distinct a.key
+                   from src a
+                   where b.value <> a.key and a.key > '9'
+                  )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSemiJoin(condition=[AND(=($0, $2), <>($1, $3))], joinType=[semi])
+  HiveProject(key=[$0], value=[$1])
+    HiveFilter(condition=[IS NOT NULL($0)])
+      HiveTableScan(table=[[default, src]], table:alias=[b])
+  HiveProject(key=[$0], key0=[$0])
+    HiveAggregate(group=[{0}])
+      HiveFilter(condition=[>($0, _UTF-16LE'9')])
+        HiveTableScan(table=[[default, src]], table:alias=[a])
+
+PREHOOK: query: select *
+from src b
+where b.key in
+      (select distinct a.key
+       from src a
+       where b.value <> a.key and a.key > '9'
+      )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+from src b
+where b.key in
+      (select distinct a.key
+       from src a
+       where b.value <> a.key and a.key > '9'
+      )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+90	val_90
+90	val_90
+90	val_90
+92	val_92
+95	val_95
+95	val_95
+96	val_96
+97	val_97
+97	val_97
+98	val_98
+98	val_98
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 9201d5b..2152c01 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
@@ -1717,7 +1717,7 @@
           HiveTableScan(table=[[default, part_null]], table:alias=[part_null])
         HiveProject(p_type=[$0], c=[$1], ck=[$2])
           HiveAggregate(group=[{1}], c=[COUNT()], ck=[COUNT($2)])
-            HiveSemiJoin(condition=[AND(=($1, $4), =($0, $3))], joinType=[semi])
+            HiveSemiJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[semi])
               HiveProject(p_brand=[$3], p_type=[$4], p_container=[$6])
                 HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($3))])
                   HiveTableScan(table=[[default, part]], table:alias=[part])
@@ -1726,7 +1726,7 @@
                   HiveTableScan(table=[[default, part]], table:alias=[pp])
       HiveProject(p_container=[$1], literalTrue=[true], p_type=[$0])
         HiveAggregate(group=[{1, 2}])
-          HiveSemiJoin(condition=[AND(=($1, $4), =($0, $3))], joinType=[semi])
+          HiveSemiJoin(condition=[AND(=($0, $3), =($1, $4))], joinType=[semi])
             HiveProject(p_brand=[$3], p_type=[$4], p_container=[$6])
               HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($3), IS NOT NULL($6))])
                 HiveTableScan(table=[[default, part]], table:alias=[part])
@@ -1770,10 +1770,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part_null
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_type is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (p_type is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_type 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)
@@ -2094,10 +2094,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part_null
-                  filterExpr: (p_name is not null and p_type is not null and p_brand is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_brand is not null and p_type 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 and p_type is not null and p_brand is not null) (type: boolean)
+                    predicate: (p_name is not null and p_brand is not null and p_type 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)
@@ -2269,10 +2269,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part_null
-                  filterExpr: (p_type is not null and p_container is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_type is not null and p_container is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (p_type is not null and p_container is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_type is not null and p_container 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)
@@ -2592,10 +2592,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part_null
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_type is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (p_type is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_type 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)
@@ -2835,10 +2835,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part_null
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_type is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (p_type is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_type 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)
@@ -3290,10 +3290,10 @@
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: ((value is not null and key is not null) or ((key > '9') and value is not null)) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or ((key > '9') and 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)
+                    predicate: (key is not null and value is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
@@ -3808,7 +3808,7 @@
 96	val_96	1
 97	val_97	2
 98	val_98	2
-Warning: Shuffle Join MERGEJOIN[55][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[51][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: explain select * from part where p_name IN (select p_name from part p where part.p_type <> '1')
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -3835,22 +3835,22 @@
             Map Operator Tree:
                 TableScan
                   alias: part
-                  filterExpr: (((p_type <> '1') and p_name is not null) or p_name is not null or (p_type <> '1')) (type: boolean)
+                  filterExpr: ((p_name is not null and p_type is not null) or p_name is not null or (p_type <> '1')) (type: boolean)
                   Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: ((p_type <> '1') and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and 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
+                      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), (p_type <> '1') (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+                      Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col1 (type: string), _col4 (type: string)
+                        key expressions: _col1 (type: string), _col9 (type: boolean)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: _col1 (type: string), _col4 (type: string)
-                        Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+                        Map-reduce partition columns: _col1 (type: string), _col9 (type: boolean)
+                        Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
+                        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)
                   Filter Operator
                     predicate: p_name is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
@@ -3866,18 +3866,22 @@
                   Filter Operator
                     predicate: (p_type <> '1') (type: boolean)
                     Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: p_type (type: string)
-                      minReductionHashAggr: 0.4
-                      mode: hash
+                    Select Operator
+                      expressions: (p_type <> '1') (type: boolean)
                       outputColumnNames: _col0
-                      Statistics: Num rows: 24 Data size: 2496 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
+                      Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: boolean)
+                        minReductionHashAggr: 0.9230769
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -3887,8 +3891,8 @@
                 condition map:
                      Left Semi Join 0 to 1
                 keys:
-                  0 _col1 (type: string), _col4 (type: string)
-                  1 _col0 (type: string), _col1 (type: string)
+                  0 _col1 (type: string), _col9 (type: boolean)
+                  1 _col0 (type: string), _col1 (type: boolean)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
@@ -3908,32 +3912,32 @@
                   0 
                   1 
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 624 Data size: 140400 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 52 Data size: 6500 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
-                  keys: _col0 (type: string), _col1 (type: string)
-                  minReductionHashAggr: 0.80448717
+                  keys: _col0 (type: string), _col1 (type: boolean)
+                  minReductionHashAggr: 0.4
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 312 Data size: 70200 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: _col0 (type: string), _col1 (type: string)
+                    key expressions: _col0 (type: string), _col1 (type: boolean)
                     null sort order: zz
                     sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 312 Data size: 70200 Basic stats: COMPLETE Column stats: COMPLETE
+                    Map-reduce partition columns: _col0 (type: string), _col1 (type: boolean)
+                    Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: string)
+                keys: KEY._col0 (type: boolean)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
                   null sort order: 
                   sort order: 
-                  Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: string)
+                  Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator
@@ -3941,7 +3945,7 @@
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join MERGEJOIN[55][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[51][tables = [$hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
 PREHOOK: query: select * from part where p_name IN (select p_name from part p where part.p_type <> '1')
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -4997,10 +5001,10 @@
             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)
+                  filterExpr: ((key is not null and value 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)
+                    predicate: (key is not null and value is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
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 6ee1101..e6690f2 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out
@@ -626,8 +626,30 @@
 #### A masked pattern was here ####
 98	val_98
 98	val_98
-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 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: explain cbo 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
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo 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)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(p_name=[$0])
+  HiveAntiJoin(condition=[=($3, $1)], joinType=[anti])
+    HiveJoin(condition=[true], joinType=[left], algorithm=[none], cost=[not available])
+      HiveProject(p_name=[$1])
+        HiveTableScan(table=[[default, part]], table:alias=[p1])
+      HiveProject(CAST=[CAST($4):DOUBLE])
+        HiveTableScan(table=[[default, part]], table:alias=[part])
+    HiveProject(literalTrue=[true], p_partkey0=[CAST($0):DOUBLE])
+      HiveFilter(condition=[IS NOT NULL(CAST($0):DOUBLE)])
+        HiveTableScan(table=[[default, part]], table:alias=[pp1])
+
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' 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
@@ -647,52 +669,15 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 7 (CUSTOM_SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (XPROD_EDGE), Map 7 (XPROD_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-        Reducer 6 <- Map 7 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
             Map Operator Tree:
                 TableScan
-                  alias: part
-                  Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: p_type (type: string)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      null sort order: 
-                      sort order: 
-                      Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: string)
-                  Filter Operator
-                    predicate: (UDFToDouble(p_type) is not null and p_type is not null) (type: boolean)
-                    Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: p_type (type: string)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        null sort order: 
-                        sort order: 
-                        Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: string)
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
                   alias: p1
-                  Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                    Reduce Output Operator
-                      null sort order: 
-                      sort order: 
-                      Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 26 Data size: 3146 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: p_name (type: string)
                     outputColumnNames: _col0
@@ -709,12 +694,34 @@
                       expressions: UDFToDouble(p_partkey) (type: double)
                       outputColumnNames: _col0
                       Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: double)
-                        Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 25 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: double)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: double)
+                          Statistics: Num rows: 25 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: part
+                  Statistics: Num rows: 26 Data size: 2704 Basic stats: COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: UDFToDouble(p_type) (type: double)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      null sort order: 
+                      sort order: 
+                      Statistics: Num rows: 26 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: double)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -727,13 +734,13 @@
                   0 
                   1 
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 676 Data size: 152100 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 676 Data size: 87204 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col1 (type: string)
+                  key expressions: _col1 (type: double)
                   null sort order: z
                   sort order: +
-                  Map-reduce partition columns: _col1 (type: string)
-                  Statistics: Num rows: 676 Data size: 152100 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map-reduce partition columns: _col1 (type: double)
+                  Statistics: Num rows: 676 Data size: 87204 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col0 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -742,8 +749,8 @@
                 condition map:
                      Anti Join 0 to 1
                 keys:
-                  0 _col1 (type: string)
-                  1 _col0 (type: string)
+                  0 _col1 (type: double)
+                  1 _col0 (type: double)
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 121 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
@@ -753,75 +760,6 @@
                       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:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 
-                  1 
-                outputColumnNames: _col1
-                Statistics: Num rows: 676 Data size: 70304 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  keys: _col1 (type: string)
-                  minReductionHashAggr: 0.964497
-                  mode: hash
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 24 Data size: 2496 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
-        Reducer 5 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: string)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 24 Data size: 2688 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 24 Data size: 2688 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: string)
-        Reducer 6 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col1
-                Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col1 (type: string)
-                  outputColumnNames: _col0
-                  Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
-                  Group By Operator
-                    keys: _col0 (type: string)
-                    minReductionHashAggr: 0.4
-                    mode: hash
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 24 Data size: 2496 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
 
   Stage: Stage-0
     Fetch Operator
@@ -829,8 +767,7 @@
       Processor Tree:
         ListSink
 
-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 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' 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
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
index e3ef547..e39320a 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notin.q.out
@@ -2286,6 +2286,31 @@
 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 cbo select count(*) as c from part as e where p_size + 100 NOT IN (select p_partkey from part where p_name = e.p_name)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select count(*) as c from part as e where p_size + 100 NOT IN (select p_partkey from part where p_name = e.p_name)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAggregate(group=[{}], agg#0=[count()])
+  HiveFilter(condition=[OR(IS NULL($2), =($2, 0), IS NOT TRUE(OR(IS NOT NULL($6), IS NULL($1), <($3, $2))))])
+    HiveProject(p_name=[$0], p_size=[$1], c=[$4], ck=[$5], $f15=[$2], p_partkey=[$6], literalTrue=[$7], p_name0=[$8])
+      HiveJoin(condition=[AND(=($8, $0), =($2, $6))], joinType=[left], algorithm=[none], cost=[not available])
+        HiveJoin(condition=[=($3, $0)], joinType=[left], algorithm=[none], cost=[not available])
+          HiveProject(p_name=[$1], p_size=[$5], +=[+($5, 100)])
+            HiveTableScan(table=[[default, part]], table:alias=[e])
+          HiveProject(p_name=[$0], c=[$1], ck=[$2])
+            HiveAggregate(group=[{1}], c=[COUNT()], ck=[COUNT($0)])
+              HiveFilter(condition=[IS NOT NULL($1)])
+                HiveTableScan(table=[[default, part]], table:alias=[part])
+        HiveProject(p_partkey=[$0], literalTrue=[true], p_name=[$1])
+          HiveAggregate(group=[{0, 1}])
+            HiveFilter(condition=[AND(IS NOT NULL($1), IS NOT NULL($0))])
+              HiveTableScan(table=[[default, part]], table:alias=[part])
+
 PREHOOK: query: explain select count(*) as c from part as e where p_size + 100 NOT IN (select p_partkey from part where p_name = e.p_name)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2304,12 +2329,10 @@
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
         Reducer 5 <- Map 1 (SIMPLE_EDGE)
         Reducer 6 <- Map 1 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-        Reducer 8 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -2318,16 +2341,16 @@
                   alias: e
                   Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: p_name (type: string), p_size (type: int)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: p_name (type: string), p_size (type: int), (p_size + 100) (type: int)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 26 Data size: 3354 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: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: int)
+                      Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: int), _col2 (type: int)
                   Filter Operator
                     predicate: p_name is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2349,32 +2372,17 @@
                     predicate: (p_name is not null and p_partkey is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 3250 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      keys: p_partkey (type: int), p_name (type: string)
+                      keys: p_name (type: string), p_partkey (type: int)
                       minReductionHashAggr: 0.4
                       mode: hash
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 25 Data size: 3125 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
-                        key expressions: _col0 (type: int), _col1 (type: string)
+                        key expressions: _col0 (type: string), _col1 (type: int)
                         null sort order: zz
                         sort order: ++
-                        Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
+                        Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
                         Statistics: Num rows: 25 Data size: 3125 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: p_size is not null (type: boolean)
-                    Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: p_size (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 21 Data size: 84 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: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -2386,15 +2394,15 @@
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
-                outputColumnNames: _col0, _col1, _col3, _col4
-                Statistics: Num rows: 26 Data size: 3666 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col1, _col2, _col4, _col5
+                Statistics: Num rows: 26 Data size: 3770 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col0 (type: string), _col1 (type: int)
+                  key expressions: _col0 (type: string), _col2 (type: int)
                   null sort order: zz
                   sort order: ++
-                  Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
-                  Statistics: Num rows: 26 Data size: 3666 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col3 (type: bigint), _col4 (type: bigint)
+                  Map-reduce partition columns: _col0 (type: string), _col2 (type: int)
+                  Statistics: Num rows: 26 Data size: 3770 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: int), _col4 (type: bigint), _col5 (type: bigint)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2402,26 +2410,30 @@
                 condition map:
                      Left Outer Join 0 to 1
                 keys:
-                  0 _col0 (type: string), _col1 (type: int)
-                  1 _col2 (type: string), _col3 (type: int)
-                outputColumnNames: _col1, _col3, _col4, _col6
-                Statistics: Num rows: 26 Data size: 612 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: (_col3 is null or (_col3 = 0L) or (_col6 is not null or _col1 is null or (_col4 < _col3)) is not true) (type: boolean)
-                  Statistics: Num rows: 14 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    Statistics: Num rows: 14 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: count()
-                      minReductionHashAggr: 0.9285714
-                      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: 
+                  0 _col0 (type: string), _col2 (type: int)
+                  1 _col2 (type: string), _col0 (type: int)
+                outputColumnNames: _col1, _col4, _col5, _col7
+                Statistics: Num rows: 26 Data size: 624 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col7 (type: boolean)
+                  outputColumnNames: _col1, _col2, _col3, _col6
+                  Statistics: Num rows: 26 Data size: 624 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (_col2 is null or (_col2 = 0L) or (_col6 is not null or _col1 is null or (_col3 < _col2)) is not true) (type: boolean)
+                    Statistics: Num rows: 14 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      Statistics: Num rows: 14 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: count()
+                        minReductionHashAggr: 0.9285714
+                        mode: hash
+                        outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: bigint)
+                        Reduce Output Operator
+                          null sort order: 
+                          sort order: 
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: bigint)
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -2457,54 +2469,21 @@
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: int), KEY._col1 (type: string)
+                keys: KEY._col0 (type: string), KEY._col1 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 25 Data size: 3125 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), true (type: boolean), _col1 (type: string)
+                  expressions: _col1 (type: int), true (type: boolean), _col0 (type: string)
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 25 Data size: 3225 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)
+                    key expressions: _col2 (type: string), _col0 (type: int)
+                    null sort order: zz
+                    sort order: ++
+                    Map-reduce partition columns: _col2 (type: string), _col0 (type: int)
                     Statistics: Num rows: 25 Data size: 3225 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean), _col2 (type: string)
-        Reducer 7 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: int)
-                  1 (_col0 + 100) (type: int)
-                outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 21 Data size: 2709 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col2 (type: string), _col3 (type: int)
-                  null sort order: zz
-                  sort order: ++
-                  Map-reduce partition columns: _col2 (type: string), _col3 (type: int)
-                  Statistics: Num rows: 21 Data size: 2709 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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: (_col0 + 100) (type: int)
-                  null sort order: z
-                  sort order: +
-                  Map-reduce partition columns: (_col0 + 100) (type: int)
-                  Statistics: Num rows: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: int)
+                    value expressions: _col1 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator
@@ -2723,6 +2702,31 @@
 POSTHOOK: Input: default@part
 #### 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
+PREHOOK: query: explain cbo select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(p_partkey=[$0], p_name=[$1], p_mfgr=[$2], p_brand=[$3], p_type=[$4], p_size=[$5], p_container=[$6], p_retailprice=[$7], p_comment=[$8])
+  HiveFilter(condition=[OR(IS NULL($9), =($9, 0), IS NOT TRUE(OR(IS NOT NULL($13), IS NULL($1), <($10, $9))))])
+    HiveProject(p_partkey=[$0], p_name=[$1], p_mfgr=[$2], p_brand=[$3], p_type=[$4], p_size=[$5], p_container=[$6], p_retailprice=[$7], p_comment=[$8], c=[$12], ck=[$13], $f17=[+($5, 121150)], p_name0=[$14], literalTrue=[$15], p_size0=[$16], p_partkey0=[$17])
+      HiveJoin(condition=[AND(=($1, $14), =($16, $5), =(+($5, 121150), $17))], joinType=[left], algorithm=[none], cost=[not available])
+        HiveJoin(condition=[AND(=($10, $5), =($9, $11))], joinType=[left], algorithm=[none], cost=[not available])
+          HiveProject(p_partkey=[$0], p_name=[$1], p_mfgr=[$2], p_brand=[$3], p_type=[$4], p_size=[$5], p_container=[$6], p_retailprice=[$7], p_comment=[$8], $f14=[+($5, 121150)])
+            HiveTableScan(table=[[default, part]], table:alias=[part])
+          HiveProject(p_size=[$1], p_partkey=[$0], c=[$2], ck=[$3])
+            HiveAggregate(group=[{0, 5}], c=[COUNT()], ck=[COUNT($1)])
+              HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($0))])
+                HiveTableScan(table=[[default, part]], table:alias=[p])
+        HiveProject(p_name=[$1], literalTrue=[true], p_size=[$2], p_partkey=[$0])
+          HiveAggregate(group=[{0, 1, 5}])
+            HiveFilter(condition=[AND(IS NOT NULL($5), IS NOT NULL($0), IS NOT NULL($1))])
+              HiveTableScan(table=[[default, part]], table:alias=[p])
+
 PREHOOK: query: explain select * from part where p_name NOT IN (select p_name from part p where p.p_size = part.p_size AND part.p_size + 121150 = p.p_partkey )
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
@@ -2752,56 +2756,48 @@
                   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
+                    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), (p_size + 121150) (type: int)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+                    Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      key expressions: _col5 (type: int)
-                      null sort order: z
-                      sort order: +
-                      Map-reduce partition columns: _col5 (type: int)
-                      Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+                      key expressions: _col5 (type: int), _col9 (type: int)
+                      null sort order: zz
+                      sort order: ++
+                      Map-reduce partition columns: _col5 (type: int), _col9 (type: int)
+                      Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
                       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)
                   Filter Operator
-                    predicate: (((p_size + 121150) = p_partkey) and p_size is not null) (type: boolean)
-                    Statistics: Num rows: 13 Data size: 1677 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: p_name (type: string), p_size (type: int)
-                      outputColumnNames: p_name, p_size
-                      Statistics: Num rows: 13 Data size: 1677 Basic stats: COMPLETE Column stats: COMPLETE
-                      Group By Operator
-                        aggregations: count(), count(p_name)
-                        keys: p_size (type: int)
-                        minReductionHashAggr: 0.4
-                        mode: hash
-                        outputColumnNames: _col0, _col1, _col2
-                        Statistics: Num rows: 13 Data size: 260 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: 13 Data size: 260 Basic stats: COMPLETE Column stats: COMPLETE
-                          value expressions: _col1 (type: bigint), _col2 (type: bigint)
+                    predicate: (p_size is not null and p_partkey is not null) (type: boolean)
+                    Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      aggregations: count(), count(p_name)
+                      keys: p_size (type: int), p_partkey (type: int)
+                      minReductionHashAggr: 0.4
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2, _col3
+                      Statistics: Num rows: 25 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: int)
+                        null sort order: zz
+                        sort order: ++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                        Statistics: Num rows: 25 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col2 (type: bigint), _col3 (type: bigint)
                   Filter Operator
-                    predicate: (((p_size + 121150) = p_partkey) and p_size is not null and p_name is not null) (type: boolean)
-                    Statistics: Num rows: 13 Data size: 1677 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: p_name (type: string), p_size (type: int)
-                      outputColumnNames: p_name, p_size
-                      Statistics: Num rows: 13 Data size: 1677 Basic stats: COMPLETE Column stats: COMPLETE
-                      Group By Operator
-                        keys: p_name (type: string), p_size (type: int)
-                        minReductionHashAggr: 0.4
-                        mode: hash
-                        outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: string), _col1 (type: int)
-                          null sort order: zz
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
-                          Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (p_size is not null and p_partkey is not null and p_name is not null) (type: boolean)
+                    Statistics: Num rows: 26 Data size: 3354 Basic stats: COMPLETE Column stats: COMPLETE
+                    Group By Operator
+                      keys: p_size (type: int), p_name (type: string), p_partkey (type: int)
+                      minReductionHashAggr: 0.4
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 25 Data size: 3225 Basic stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int)
+                        null sort order: zzz
+                        sort order: +++
+                        Map-reduce partition columns: _col0 (type: int), _col1 (type: string), _col2 (type: int)
+                        Statistics: Num rows: 25 Data size: 3225 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -2811,17 +2807,17 @@
                 condition map:
                      Left Outer Join 0 to 1
                 keys:
-                  0 _col5 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11
-                Statistics: Num rows: 42 Data size: 26270 Basic stats: COMPLETE Column stats: COMPLETE
+                  0 _col5 (type: int), _col9 (type: int)
+                  1 _col0 (type: int), _col1 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col12, _col13
+                Statistics: Num rows: 30 Data size: 19002 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col1 (type: string), _col5 (type: int)
-                  null sort order: zz
-                  sort order: ++
-                  Map-reduce partition columns: _col1 (type: string), _col5 (type: int)
-                  Statistics: Num rows: 42 Data size: 26270 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: bigint), _col11 (type: bigint)
+                  key expressions: _col5 (type: int), _col1 (type: string), (_col5 + 121150) (type: int)
+                  null sort order: zzz
+                  sort order: +++
+                  Map-reduce partition columns: _col5 (type: int), _col1 (type: string), (_col5 + 121150) (type: int)
+                  Statistics: Num rows: 30 Data size: 19002 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col12 (type: bigint), _col13 (type: bigint)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -2829,58 +2825,62 @@
                 condition map:
                      Left Outer Join 0 to 1
                 keys:
-                  0 _col1 (type: string), _col5 (type: int)
-                  1 _col0 (type: string), _col2 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col13
-                Statistics: Num rows: 63 Data size: 39693 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: (_col10 is null or (_col10 = 0L) or (_col13 is not null or _col1 is null or (_col11 < _col10)) is not true) (type: boolean)
-                  Statistics: Num rows: 62 Data size: 39058 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: 62 Data size: 38378 Basic stats: COMPLETE Column stats: COMPLETE
-                    File Output Operator
-                      compressed: false
-                      Statistics: Num rows: 62 Data size: 38378 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
+                  0 _col5 (type: int), _col1 (type: string), (_col5 + 121150) (type: int)
+                  1 _col2 (type: int), _col0 (type: string), _col3 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col12, _col13, _col15
+                Statistics: Num rows: 30 Data size: 19122 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), _col12 (type: bigint), _col13 (type: bigint), _col15 (type: boolean)
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col13
+                  Statistics: Num rows: 30 Data size: 19122 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (_col9 is null or (_col9 = 0L) or (_col13 is not null or _col1 is null or (_col10 < _col9)) is not true) (type: boolean)
+                    Statistics: Num rows: 20 Data size: 12748 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: 20 Data size: 12380 Basic stats: COMPLETE Column stats: COMPLETE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 20 Data size: 12380 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:
               Group By Operator
                 aggregations: count(VALUE._col0), count(VALUE._col1)
-                keys: KEY._col0 (type: int)
+                keys: KEY._col0 (type: int), KEY._col1 (type: int)
                 mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 13 Data size: 260 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 25 Data size: 600 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: 13 Data size: 260 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: bigint), _col2 (type: bigint)
+                  key expressions: _col0 (type: int), _col1 (type: int)
+                  null sort order: zz
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                  Statistics: Num rows: 25 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col2 (type: bigint), _col3 (type: bigint)
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: string), KEY._col1 (type: int)
+                keys: KEY._col0 (type: int), KEY._col1 (type: string), KEY._col2 (type: int)
                 mode: mergepartial
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 13 Data size: 1625 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 25 Data size: 3225 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: string), true (type: boolean), _col1 (type: int)
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 13 Data size: 1677 Basic stats: COMPLETE Column stats: COMPLETE
+                  expressions: _col1 (type: string), true (type: boolean), _col0 (type: int), _col2 (type: int)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 25 Data size: 3325 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: _col0 (type: string), _col2 (type: int)
-                    null sort order: zz
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: string), _col2 (type: int)
-                    Statistics: Num rows: 13 Data size: 1677 Basic stats: COMPLETE Column stats: COMPLETE
+                    key expressions: _col2 (type: int), _col0 (type: string), _col3 (type: int)
+                    null sort order: zzz
+                    sort order: +++
+                    Map-reduce partition columns: _col2 (type: int), _col0 (type: string), _col3 (type: int)
+                    Statistics: Num rows: 25 Data size: 3325 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: boolean)
 
   Stage: Stage-0
@@ -3278,12 +3278,10 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Map 1 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-        Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
-        Reducer 9 <- Map 7 (SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -3303,34 +3301,15 @@
                       Statistics: Num rows: 26 Data size: 5954 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: string), _col1 (type: string)
                   Filter Operator
-                    predicate: (p_size is not null and p_type is not null) (type: boolean)
-                    Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
-                    Select Operator
-                      expressions: p_type (type: string), (p_size + 1) (type: int)
-                      outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
-                      Group By Operator
-                        keys: _col1 (type: int), _col0 (type: string)
-                        minReductionHashAggr: 0.4
-                        mode: hash
-                        outputColumnNames: _col0, _col1
-                        Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
-                        Reduce Output Operator
-                          key expressions: _col0 (type: int), _col1 (type: string)
-                          null sort order: zz
-                          sort order: ++
-                          Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
-                          Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
                     predicate: p_size is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: p_type (type: string), (p_size + 1) (type: int)
+                      expressions: (p_size + 1) (type: int), p_type (type: string)
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: count(), count(_col0)
-                        keys: _col1 (type: int)
+                        aggregations: count(), count(_col1)
+                        keys: _col0 (type: int)
                         minReductionHashAggr: 0.4
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2
@@ -3342,39 +3321,25 @@
                           Map-reduce partition columns: _col0 (type: int)
                           Statistics: Num rows: 21 Data size: 420 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: bigint), _col2 (type: bigint)
-            Execution mode: vectorized, llap
-            LLAP IO: all inputs
-        Map 7 
-            Map Operator Tree:
-                TableScan
-                  alias: part
-                  filterExpr: p_size is not null (type: boolean)
-                  Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: p_size is not null (type: boolean)
-                    Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (p_size is not null and p_type is not null) (type: boolean)
+                    Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: (p_size + 1) (type: int)
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+                      expressions: p_type (type: string), (p_size + 1) (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 26 Data size: 2808 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        keys: _col0 (type: int)
+                        keys: _col0 (type: string), _col1 (type: int)
                         minReductionHashAggr: 0.4
                         mode: hash
-                        outputColumnNames: _col0
-                        Statistics: Num rows: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
+                        outputColumnNames: _col0, _col1
+                        Statistics: Num rows: 24 Data size: 2592 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: 21 Data size: 84 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: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
+                          key expressions: _col0 (type: string), _col1 (type: int)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
+                          Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -3424,34 +3389,28 @@
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: int), KEY._col1 (type: string)
+                aggregations: count(VALUE._col0), count(VALUE._col1)
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 21 Data size: 420 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: 21 Data size: 420 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: bigint), _col2 (type: bigint)
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: string), KEY._col1 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col1 (type: string), _col0 (type: int)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: int)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: int)
-                    Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: string)
-        Reducer 5 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: int)
-                  1 _col0 (type: int)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 24 Data size: 2592 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: string), true (type: boolean), _col2 (type: int)
+                  expressions: _col0 (type: string), true (type: boolean), _col1 (type: int)
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 24 Data size: 2688 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
@@ -3461,55 +3420,6 @@
                     Map-reduce partition columns: _col0 (type: string), _col2 (type: int)
                     Statistics: Num rows: 24 Data size: 2688 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: boolean)
-        Reducer 6 
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
-                Dummy Store
-            Execution mode: llap
-            Reduce Operator Tree:
-              Group By Operator
-                aggregations: count(VALUE._col0), count(VALUE._col1)
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 21 Data size: 420 Basic stats: COMPLETE Column stats: COMPLETE
-                Merge Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col1, _col2, _col3
-                  Statistics: Num rows: 21 Data size: 420 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col3 (type: int), _col1 (type: bigint), _col2 (type: bigint)
-                    outputColumnNames: _col0, _col1, _col2
-                    Statistics: Num rows: 21 Data size: 420 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: 21 Data size: 420 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: bigint), _col2 (type: bigint)
-        Reducer 9 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 21 Data size: 84 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: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
 
   Stage: Stage-0
     Fetch Operator
@@ -5354,12 +5264,10 @@
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
         Reducer 5 <- Map 1 (SIMPLE_EDGE)
         Reducer 6 <- Map 1 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-        Reducer 8 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -5368,16 +5276,16 @@
                   alias: e
                   Statistics: Num rows: 26 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: p_brand (type: string), p_size (type: int)
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 26 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: p_brand (type: string), p_size (type: int), (p_size + 100) (type: int)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 26 Data size: 2600 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: 26 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col1 (type: int)
+                      Statistics: Num rows: 26 Data size: 2600 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: int), _col2 (type: int)
                   Filter Operator
                     predicate: p_brand is not null (type: boolean)
                     Statistics: Num rows: 26 Data size: 5096 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5396,7 +5304,7 @@
                         Statistics: Num rows: 16 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint), _col2 (type: bigint)
                   Filter Operator
-                    predicate: (p_brand is not null and UDFToDouble(p_type) is not null) (type: boolean)
+                    predicate: (p_brand is not null and p_type is not null and UDFToDouble(p_type) is not null) (type: boolean)
                     Statistics: Num rows: 26 Data size: 5096 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       keys: p_brand (type: string), p_type (type: string)
@@ -5410,21 +5318,6 @@
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 24 Data size: 4704 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: (UDFToDouble((p_size + 100)) is not null and p_size is not null) (type: boolean)
-                    Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: p_size (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 21 Data size: 84 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: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -5436,15 +5329,15 @@
                 keys:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
-                outputColumnNames: _col0, _col1, _col3, _col4
-                Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
+                outputColumnNames: _col0, _col1, _col2, _col4, _col5
+                Statistics: Num rows: 26 Data size: 3016 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col0 (type: string), _col1 (type: int)
+                  key expressions: _col0 (type: string), UDFToDouble(_col2) (type: double)
                   null sort order: zz
                   sort order: ++
-                  Map-reduce partition columns: _col0 (type: string), _col1 (type: int)
-                  Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col3 (type: bigint), _col4 (type: bigint)
+                  Map-reduce partition columns: _col0 (type: string), UDFToDouble(_col2) (type: double)
+                  Statistics: Num rows: 26 Data size: 3016 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: int), _col4 (type: bigint), _col5 (type: bigint)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -5452,26 +5345,30 @@
                 condition map:
                      Left Outer Join 0 to 1
                 keys:
-                  0 _col0 (type: string), _col1 (type: int)
-                  1 _col2 (type: string), _col3 (type: int)
-                outputColumnNames: _col1, _col3, _col4, _col6
-                Statistics: Num rows: 26 Data size: 624 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: (_col3 is null or (_col3 = 0L) or (_col6 is not null or _col1 is null or (_col4 < _col3)) is not true) (type: boolean)
-                  Statistics: Num rows: 15 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    Statistics: Num rows: 15 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      aggregations: count()
-                      minReductionHashAggr: 0.93333334
-                      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: 
+                  0 _col0 (type: string), UDFToDouble(_col2) (type: double)
+                  1 _col2 (type: string), UDFToDouble(_col0) (type: double)
+                outputColumnNames: _col1, _col4, _col5, _col7
+                Statistics: Num rows: 29 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col1 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col7 (type: boolean)
+                  outputColumnNames: _col1, _col2, _col3, _col6
+                  Statistics: Num rows: 29 Data size: 688 Basic stats: COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (_col2 is null or (_col2 = 0L) or (_col6 is not null or _col1 is null or (_col3 < _col2)) is not true) (type: boolean)
+                    Statistics: Num rows: 16 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      Statistics: Num rows: 16 Data size: 380 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: count()
+                        minReductionHashAggr: 0.9375
+                        mode: hash
+                        outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: bigint)
+                        Reduce Output Operator
+                          null sort order: 
+                          sort order: 
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: bigint)
         Reducer 4 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
@@ -5516,45 +5413,12 @@
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 24 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: UDFToDouble(_col0) (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: UDFToDouble(_col0) (type: double)
+                    key expressions: _col2 (type: string), UDFToDouble(_col0) (type: double)
+                    null sort order: zz
+                    sort order: ++
+                    Map-reduce partition columns: _col2 (type: string), UDFToDouble(_col0) (type: double)
                     Statistics: Num rows: 24 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean), _col2 (type: string)
-        Reducer 7 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 UDFToDouble(_col0) (type: double)
-                  1 UDFToDouble((_col0 + 100)) (type: double)
-                outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 21 Data size: 2100 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col2 (type: string), _col3 (type: int)
-                  null sort order: zz
-                  sort order: ++
-                  Map-reduce partition columns: _col2 (type: string), _col3 (type: int)
-                  Statistics: Num rows: 21 Data size: 2100 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: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: UDFToDouble((_col0 + 100)) (type: double)
-                  null sort order: z
-                  sort order: +
-                  Map-reduce partition columns: UDFToDouble((_col0 + 100)) (type: double)
-                  Statistics: Num rows: 21 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col0 (type: int)
+                    value expressions: _col1 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator
@@ -5781,6 +5645,35 @@
 POSTHOOK: Input: default@t1_n0
 POSTHOOK: Input: default@t2_n0
 #### A masked pattern was here ####
+PREHOOK: query: explain cbo SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0 where t1_n0.c2=t2_n0.c1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n0
+PREHOOK: Input: default@t2_n0
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0 where t1_n0.c2=t2_n0.c1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n0
+POSTHOOK: Input: default@t2_n0
+#### A masked pattern was here ####
+CBO PLAN:
+HiveProject(c1=[$0])
+  HiveFilter(condition=[OR(IS NULL($1), =($1, 0), IS NOT TRUE(OR(IS NOT NULL($5), IS NULL($0), <($2, $1))))])
+    HiveProject(c1=[$0], c=[$4], ck=[$5], c20=[CAST($1):DOUBLE], c10=[$6], literalTrue=[$7], c100=[$8])
+      HiveJoin(condition=[AND(=($0, $6), =(CAST($1):DOUBLE, $8))], joinType=[left], algorithm=[none], cost=[not available])
+        HiveJoin(condition=[=($2, $3)], joinType=[left], algorithm=[none], cost=[not available])
+          HiveProject(c1=[$0], c2=[$1], c20=[CAST($1):DOUBLE])
+            HiveTableScan(table=[[default, t1_n0]], table:alias=[t1_n0])
+          HiveProject(c10=[$0], c=[$1], ck=[$2])
+            HiveAggregate(group=[{0}], c=[COUNT()], ck=[COUNT($1)])
+              HiveProject(c10=[CAST($0):DOUBLE], c1=[$0])
+                HiveFilter(condition=[IS NOT NULL(CAST($0):DOUBLE)])
+                  HiveTableScan(table=[[default, t2_n0]], table:alias=[t2_n0])
+        HiveProject(c1=[$0], literalTrue=[true], c10=[$1])
+          HiveAggregate(group=[{0, 1}])
+            HiveProject(c1=[$0], c10=[CAST($0):DOUBLE])
+              HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL(CAST($0):DOUBLE))])
+                HiveTableScan(table=[[default, t2_n0]], table:alias=[t2_n0])
+
 PREHOOK: query: explain SELECT c1 FROM t1_n0 WHERE c1 NOT IN (SELECT c1 FROM t2_n0 where t1_n0.c2=t2_n0.c1)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@t1_n0
@@ -5800,13 +5693,10 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE)
-        Reducer 5 <- Map 9 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
-        Reducer 7 <- Map 9 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 6 <- Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -5815,34 +5705,19 @@
                   alias: t1_n0
                   Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
-                    expressions: c1 (type: int), c2 (type: char(100))
-                    outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
+                    expressions: c1 (type: int), c2 (type: char(100)), UDFToDouble(c2) (type: double)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
-                      key expressions: _col1 (type: char(100))
+                      key expressions: _col2 (type: double)
                       null sort order: z
                       sort order: +
-                      Map-reduce partition columns: _col1 (type: char(100))
-                      Statistics: Num rows: 4 Data size: 368 Basic stats: COMPLETE Column stats: COMPLETE
-                      value expressions: _col0 (type: int)
-                  Filter Operator
-                    predicate: (c2 is not null and UDFToDouble(c2) is not null) (type: boolean)
-                    Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: c2 (type: char(100))
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col0 (type: char(100))
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col0 (type: char(100))
-                        Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
+                      Map-reduce partition columns: _col2 (type: double)
+                      Statistics: Num rows: 4 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: char(100))
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 9 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: t2_n0
@@ -5852,30 +5727,42 @@
                     predicate: UDFToDouble(c1) is not null (type: boolean)
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
-                      expressions: c1 (type: int), UDFToDouble(c1) (type: double)
+                      expressions: UDFToDouble(c1) (type: double), c1 (type: int)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
-                        Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: int)
+                      Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: count(), count(_col1)
+                        keys: _col0 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1, _col2
+                        Statistics: Num rows: 2 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: double)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: double)
+                          Statistics: Num rows: 2 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
+                          value expressions: _col1 (type: bigint), _col2 (type: bigint)
                   Filter Operator
                     predicate: (c1 is not null and UDFToDouble(c1) is not null) (type: boolean)
                     Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: c1 (type: int), UDFToDouble(c1) (type: double)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: double)
-                        null sort order: z
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: double)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: int), _col1 (type: double)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                        value expressions: _col0 (type: int)
+                        Reduce Output Operator
+                          key expressions: _col0 (type: int), _col1 (type: double)
+                          null sort order: zz
+                          sort order: ++
+                          Map-reduce partition columns: _col0 (type: int), _col1 (type: double)
+                          Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -5885,17 +5772,17 @@
                 condition map:
                      Left Outer Join 0 to 1
                 keys:
-                  0 _col1 (type: char(100))
-                  1 _col0 (type: char(100))
-                outputColumnNames: _col0, _col1, _col3, _col4
+                  0 _col2 (type: double)
+                  1 _col0 (type: double)
+                outputColumnNames: _col0, _col1, _col4, _col5
                 Statistics: Num rows: 4 Data size: 416 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col0 (type: int), _col1 (type: char(100))
+                  key expressions: _col0 (type: int), UDFToDouble(_col1) (type: double)
                   null sort order: zz
                   sort order: ++
-                  Map-reduce partition columns: _col0 (type: int), _col1 (type: char(100))
+                  Map-reduce partition columns: _col0 (type: int), UDFToDouble(_col1) (type: double)
                   Statistics: Num rows: 4 Data size: 416 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col3 (type: bigint), _col4 (type: bigint)
+                  value expressions: _col4 (type: bigint), _col5 (type: bigint)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
@@ -5903,133 +5790,63 @@
                 condition map:
                      Left Outer Join 0 to 1
                 keys:
-                  0 _col0 (type: int), _col1 (type: char(100))
-                  1 _col0 (type: int), _col2 (type: char(100))
-                outputColumnNames: _col0, _col3, _col4, _col6
+                  0 _col0 (type: int), UDFToDouble(_col1) (type: double)
+                  1 _col0 (type: int), _col2 (type: double)
+                outputColumnNames: _col0, _col4, _col5, _col7
                 Statistics: Num rows: 5 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
-                Filter Operator
-                  predicate: (_col3 is null or (_col3 = 0L) or (_col6 is not null or _col0 is null or (_col4 < _col3)) is not true) (type: boolean)
+                Select Operator
+                  expressions: _col0 (type: int), _col4 (type: bigint), _col5 (type: bigint), _col7 (type: boolean)
+                  outputColumnNames: _col0, _col1, _col2, _col5
                   Statistics: Num rows: 5 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
-                  Select Operator
-                    expressions: _col0 (type: int)
-                    outputColumnNames: _col0
-                    Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
-                    File Output Operator
-                      compressed: false
+                  Filter Operator
+                    predicate: (_col1 is null or (_col1 = 0L) or (_col5 is not null or _col0 is null or (_col2 < _col1)) is not true) (type: boolean)
+                    Statistics: Num rows: 5 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: _col0 (type: int)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 5 Data size: 20 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:
-              Group By Operator
-                keys: KEY._col0 (type: char(100))
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 2 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: char(100)), UDFToDouble(_col0) (type: double)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 2 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 2 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: char(100))
-                  Reduce Output Operator
-                    key expressions: _col1 (type: double)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col1 (type: double)
-                    Statistics: Num rows: 2 Data size: 192 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: char(100))
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 5 Data size: 20 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:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  keys: _col2 (type: int), _col0 (type: char(100))
-                  minReductionHashAggr: 0.4
-                  mode: hash
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int), _col1 (type: char(100))
-                    null sort order: zz
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: int), _col1 (type: char(100))
-                    Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int), KEY._col1 (type: char(100))
-                mode: mergepartial
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), true (type: boolean), _col1 (type: char(100))
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: int), _col2 (type: char(100))
-                    null sort order: zz
-                    sort order: ++
-                    Map-reduce partition columns: _col0 (type: int), _col2 (type: char(100))
-                    Statistics: Num rows: 1 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
-        Reducer 7 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col1 (type: double)
-                  1 _col1 (type: double)
-                outputColumnNames: _col0, _col2
-                Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
-                Group By Operator
-                  aggregations: count(), count(_col2)
-                  keys: _col0 (type: char(100))
-                  minReductionHashAggr: 0.4
-                  mode: hash
-                  outputColumnNames: _col0, _col1, _col2
-                  Statistics: Num rows: 2 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col0 (type: char(100))
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col0 (type: char(100))
-                    Statistics: Num rows: 2 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: bigint), _col2 (type: bigint)
-        Reducer 8 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0), count(VALUE._col1)
-                keys: KEY._col0 (type: char(100))
+                keys: KEY._col0 (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 2 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col0 (type: char(100))
+                  key expressions: _col0 (type: double)
                   null sort order: z
                   sort order: +
-                  Map-reduce partition columns: _col0 (type: char(100))
-                  Statistics: Num rows: 2 Data size: 208 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map-reduce partition columns: _col0 (type: double)
+                  Statistics: Num rows: 2 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint), _col2 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int), KEY._col1 (type: double)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), true (type: boolean), _col1 (type: double)
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int), _col2 (type: double)
+                    null sort order: zz
+                    sort order: ++
+                    Map-reduce partition columns: _col0 (type: int), _col2 (type: double)
+                    Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col1 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator
@@ -6352,12 +6169,10 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-        Reducer 7 <- Map 6 (SIMPLE_EDGE)
-        Reducer 8 <- Map 6 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 6 <- Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -6376,28 +6191,13 @@
                       Map-reduce partition columns: _col1 (type: int)
                       Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int)
-                  Filter Operator
-                    predicate: j is not null (type: boolean)
-                    Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: j (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 6 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: t7
-                  filterExpr: (j is not null or (i is not null and j is not null)) (type: boolean)
+                  filterExpr: (j is not null or (j = i)) (type: boolean)
                   Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: j is not null (type: boolean)
@@ -6417,10 +6217,10 @@
                         Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint), _col2 (type: bigint)
                   Filter Operator
-                    predicate: (i is not null and j is not null) (type: boolean)
+                    predicate: (j = i) (type: boolean)
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      keys: i (type: int), j (type: int)
+                      keys: j (type: int), i (type: int)
                       minReductionHashAggr: 0.4
                       mode: hash
                       outputColumnNames: _col0, _col1
@@ -6459,11 +6259,11 @@
                      Left Outer Join 0 to 1
                 keys:
                   0 _col1 (type: int)
-                  1 _col2 (type: int)
-                outputColumnNames: _col0, _col1, _col3, _col4, _col6
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col1, _col3, _col4, _col5
                 Statistics: Num rows: 2 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
-                  predicate: (_col3 is null or (_col3 = 0L) or (_col6 is not null or _col1 is null or (_col4 < _col3)) is not true) (type: boolean)
+                  predicate: (_col3 is null or (_col3 = 0L) or (_col5 is not null or _col1 is null or (_col4 < _col3)) is not true) (type: boolean)
                   Statistics: Num rows: 2 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: int), _col1 (type: int)
@@ -6476,39 +6276,7 @@
                           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: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 5 
-            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: _col1, _col2
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col2 (type: int)
-                  null sort order: z
-                  sort order: +
-                  Map-reduce partition columns: _col2 (type: int)
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: boolean)
-        Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -6524,7 +6292,7 @@
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint), _col2 (type: bigint)
-        Reducer 8 
+        Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -6533,16 +6301,16 @@
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), true (type: boolean)
+                  expressions: true (type: boolean), _col0 (type: int)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: _col0 (type: int)
+                    key expressions: _col1 (type: int)
                     null sort order: z
                     sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
+                    Map-reduce partition columns: _col1 (type: int)
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
+                    value expressions: _col0 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator
@@ -6613,11 +6381,9 @@
 #### A masked pattern was here ####
       Edges:
         Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
         Reducer 4 <- Map 1 (SIMPLE_EDGE)
         Reducer 5 <- Map 1 (SIMPLE_EDGE)
-        Reducer 6 <- Reducer 5 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 7 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -6653,33 +6419,21 @@
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint), _col2 (type: bigint)
-                    Group By Operator
-                      keys: j (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
-                    predicate: (j is not null and i is not null) (type: boolean)
-                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                    predicate: (j = i) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      keys: i (type: int), j (type: int)
+                      keys: j (type: int), i (type: int)
                       minReductionHashAggr: 0.4
                       mode: hash
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: int)
                         null sort order: zz
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
-                        Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -6708,11 +6462,11 @@
                      Left Outer Join 0 to 1
                 keys:
                   0 _col1 (type: int)
-                  1 _col2 (type: int)
-                outputColumnNames: _col0, _col1, _col3, _col4, _col6
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col1, _col3, _col4, _col5
                 Statistics: Num rows: 3 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
-                  predicate: (_col3 is null or (_col3 = 0L) or (_col6 is not null or _col1 is null or (_col4 < _col3)) is not true) (type: boolean)
+                  predicate: (_col3 is null or (_col3 = 0L) or (_col5 is not null or _col1 is null or (_col4 < _col3)) is not true) (type: boolean)
                   Statistics: Num rows: 3 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: int)
@@ -6745,53 +6499,21 @@
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 6 
-            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: _col1, _col2
-                Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col2 (type: int)
-                  null sort order: z
-                  sort order: +
-                  Map-reduce partition columns: _col2 (type: int)
-                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: boolean)
-        Reducer 7 
-            Execution mode: vectorized, llap
-            Reduce Operator Tree:
-              Group By Operator
                 keys: KEY._col0 (type: int), KEY._col1 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), true (type: boolean)
+                  expressions: true (type: boolean), _col0 (type: int)
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: _col0 (type: int)
+                    key expressions: _col1 (type: int)
                     null sort order: z
                     sort order: +
-                    Map-reduce partition columns: _col0 (type: int)
-                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
+                    Map-reduce partition columns: _col1 (type: int)
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notin_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_notin_having.q.out
index ea1aa3a..90531d6 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notin_having.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notin_having.q.out
@@ -912,9 +912,9 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 6 <- Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -938,24 +938,9 @@
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: c1 is not null (type: boolean)
-                    Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: c1 (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 3 Data size: 12 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: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: t2_n85
@@ -965,18 +950,6 @@
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      keys: c1 (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
                       aggregations: count(), count(c1)
                       keys: c1 (type: int)
                       minReductionHashAggr: 0.4
@@ -990,6 +963,18 @@
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint), _col2 (type: bigint)
+                    Group By Operator
+                      keys: c1 (type: int)
+                      minReductionHashAggr: 0.4
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -1031,11 +1016,11 @@
                      Left Outer Join 0 to 1
                 keys:
                   0 _col0 (type: int)
-                  1 _col2 (type: int)
-                outputColumnNames: _col0, _col2, _col3, _col5
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col2, _col3, _col4
                 Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
-                  predicate: (_col2 is null or (_col2 = 0L) or (_col5 is not null or _col0 is null or (_col3 < _col2)) is not true) (type: boolean)
+                  predicate: (_col2 is null or (_col2 = 0L) or (_col4 is not null or _col0 is null or (_col3 < _col2)) is not true) (type: boolean)
                   Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: int)
@@ -1048,7 +1033,8 @@
                           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 
+        Reducer 6 
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 keys: KEY._col0 (type: int)
@@ -1056,32 +1042,16 @@
                 outputColumnNames: _col0
                 Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: int), true (type: boolean)
+                  expressions: true (type: boolean), _col0 (type: int)
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Dummy Store
-            Execution mode: llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
-                Merge Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col1, _col2
-                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
-                    key expressions: _col2 (type: int)
+                    key expressions: _col1 (type: int)
                     null sort order: z
                     sort order: +
-                    Map-reduce partition columns: _col2 (type: int)
+                    Map-reduce partition columns: _col1 (type: int)
                     Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
+                    value expressions: _col0 (type: boolean)
 
   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 27c8311..f6fdf6b 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
@@ -4317,10 +4317,10 @@
             Map Operator Tree:
                 TableScan
                   alias: part_null_n0
-                  filterExpr: (p_type is not null and p_name is not null) (type: boolean)
+                  filterExpr: (p_name is not null and p_type is not null) (type: boolean)
                   Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (p_type is not null and p_name is not null) (type: boolean)
+                    predicate: (p_name is not null and p_type 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_n11 (type: string)
diff --git a/ql/src/test/results/clientpositive/llap/subquery_select.q.out b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
index fb971d5..dcb7a03 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_select.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_select.q.out
@@ -1806,7 +1806,7 @@
                         Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: string)
                   Filter Operator
-                    predicate: (value is not null and key is not null) (type: boolean)
+                    predicate: (key is not null and value is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
@@ -1959,10 +1959,10 @@
             Map Operator Tree:
                 TableScan
                   alias: b
-                  filterExpr: ((value is not null and key is not null) or value is not null or ((key > '9') and value is not null)) (type: boolean)
+                  filterExpr: ((key is not null and value is not null) or value is not null or ((key > '9') and 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)
+                    predicate: (key is not null and value is not null) (type: boolean)
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Select Operator
                       expressions: key (type: string), value (type: string)
diff --git a/ql/src/test/results/clientpositive/llap/subquery_views.q.out b/ql/src/test/results/clientpositive/llap/subquery_views.q.out
index 158dff1..f191a6b 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_views.q.out
@@ -108,14 +108,12 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-        Reducer 5 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
         Reducer 6 <- Map 1 (SIMPLE_EDGE)
-        Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
-        Reducer 8 <- Map 1 (SIMPLE_EDGE)
-        Reducer 9 <- Map 1 (SIMPLE_EDGE)
+        Reducer 7 <- Map 1 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -139,18 +137,6 @@
                         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: key (type: string)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 105 Data size: 9135 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: 105 Data size: 9135 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: ((key < '11') and (value > 'val_11')) (type: boolean)
                     Statistics: Num rows: 55 Data size: 9790 Basic stats: COMPLETE Column stats: COMPLETE
@@ -215,22 +201,22 @@
                      Left Outer Join 0 to 1
                 keys:
                   0 _col0 (type: string), _col1 (type: string)
-                  1 _col3 (type: string), _col2 (type: string)
-                outputColumnNames: _col0, _col1, _col4, _col5, _col7
-                Statistics: Num rows: 402 Data size: 76156 Basic stats: COMPLETE Column stats: COMPLETE
+                  1 _col1 (type: string), _col2 (type: string)
+                outputColumnNames: _col0, _col1, _col4, _col5, _col6
+                Statistics: Num rows: 201 Data size: 37158 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
-                  predicate: (_col4 is null or (_col4 = 0L) or (_col7 is not null or _col0 is null or (_col5 < _col4)) is not true) (type: boolean)
-                  Statistics: Num rows: 378 Data size: 71612 Basic stats: COMPLETE Column stats: COMPLETE
+                  predicate: (_col4 is null or (_col4 = 0L) or (_col6 is not null or _col0 is null or (_col5 < _col4)) is not true) (type: boolean)
+                  Statistics: Num rows: 201 Data size: 37158 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: _col0, _col1
-                    Statistics: Num rows: 378 Data size: 67284 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 201 Data size: 35778 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: 378 Data size: 67284 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 201 Data size: 35778 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: string)
         Reducer 4 
             Execution mode: llap
@@ -242,10 +228,10 @@
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 378 Data size: 67284 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 201 Data size: 35778 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 378 Data size: 67284 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 201 Data size: 35778 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -258,87 +244,55 @@
                      Left Outer Join 0 to 1
                 keys:
                   0 _col0 (type: string), _col1 (type: string)
-                  1 _col3 (type: string), _col2 (type: string)
-                outputColumnNames: _col0, _col4, _col5, _col7
-                Statistics: Num rows: 316 Data size: 30716 Basic stats: COMPLETE Column stats: COMPLETE
+                  1 _col1 (type: string), _col2 (type: string)
+                outputColumnNames: _col0, _col4, _col5, _col6
+                Statistics: Num rows: 201 Data size: 18867 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
-                  predicate: (_col4 is null or (_col4 = 0L) or (_col7 is not null or _col0 is null or (_col5 < _col4)) is not true) (type: boolean)
-                  Statistics: Num rows: 316 Data size: 30716 Basic stats: COMPLETE Column stats: COMPLETE
+                  predicate: (_col4 is null or (_col4 = 0L) or (_col6 is not null or _col0 is null or (_col5 < _col4)) is not true) (type: boolean)
+                  Statistics: Num rows: 201 Data size: 18867 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: string)
                     outputColumnNames: _col0
-                    Statistics: Num rows: 316 Data size: 27492 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 201 Data size: 17487 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
                       keys: _col0 (type: string)
-                      minReductionHashAggr: 0.96202534
+                      minReductionHashAggr: 0.82587063
                       mode: hash
                       outputColumnNames: _col0
-                      Statistics: Num rows: 12 Data size: 1044 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 35 Data size: 3045 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: 1044 Basic stats: COMPLETE Column stats: COMPLETE
+                        Statistics: Num rows: 35 Data size: 3045 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 6 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                keys: KEY._col0 (type: string)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 105 Data size: 9135 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: 105 Data size: 9135 Basic stats: COMPLETE Column stats: COMPLETE
-        Reducer 7 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Merge Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                keys:
-                  0 _col0 (type: string)
-                  1 _col0 (type: string)
-                outputColumnNames: _col1, _col2, _col3
-                Statistics: Num rows: 35 Data size: 6370 Basic stats: COMPLETE Column stats: COMPLETE
-                Reduce Output Operator
-                  key expressions: _col3 (type: string), _col2 (type: string)
-                  null sort order: zz
-                  sort order: ++
-                  Map-reduce partition columns: _col3 (type: string), _col2 (type: string)
-                  Statistics: Num rows: 35 Data size: 6370 Basic stats: COMPLETE Column stats: COMPLETE
-                  value expressions: _col1 (type: boolean)
-                Reduce Output Operator
-                  key expressions: _col3 (type: string), _col2 (type: string)
-                  null sort order: zz
-                  sort order: ++
-                  Map-reduce partition columns: _col3 (type: string), _col2 (type: string)
-                  Statistics: Num rows: 35 Data size: 6370 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: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 35 Data size: 6230 Basic stats: COMPLETE Column stats: COMPLETE
                 Select Operator
-                  expressions: _col0 (type: string), true (type: boolean), _col1 (type: string)
+                  expressions: true (type: boolean), _col0 (type: string), _col1 (type: string)
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 35 Data size: 6370 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)
+                    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: 35 Data size: 6370 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean), _col2 (type: string)
-        Reducer 9 
+                    value expressions: _col0 (type: boolean)
+                  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: 35 Data size: 6370 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: boolean)
+        Reducer 7 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
diff --git a/ql/src/test/results/clientpositive/llap/tez_smb_reduce_side.q.out b/ql/src/test/results/clientpositive/llap/tez_smb_reduce_side.q.out
index fe81289..34864c1 100644
--- a/ql/src/test/results/clientpositive/llap/tez_smb_reduce_side.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_smb_reduce_side.q.out
@@ -605,10 +605,10 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 7 <- Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 6 <- Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -626,24 +626,9 @@
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: c1 is not null (type: boolean)
-                    Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: c1 (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 4 Data size: 16 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: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: t2
@@ -653,18 +638,6 @@
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      keys: c1 (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
                       aggregations: count(), count(c1)
                       keys: c1 (type: int)
                       minReductionHashAggr: 0.4
@@ -678,6 +651,18 @@
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint), _col2 (type: bigint)
+                    Group By Operator
+                      keys: c1 (type: int)
+                      minReductionHashAggr: 0.4
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -706,58 +691,24 @@
                      Left Outer Join 0 to 1
                 keys:
                   0 _col0 (type: int)
-                  1 _col2 (type: int)
-                outputColumnNames: _col0, _col2, _col3, _col5
-                Statistics: Num rows: 9 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col2, _col3, _col4
+                Statistics: Num rows: 6 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
-                  predicate: (_col2 is null or (_col2 = 0L) or (_col5 is not null or _col0 is null or (_col3 < _col2)) is not true) (type: boolean)
-                  Statistics: Num rows: 9 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  predicate: (_col2 is null or (_col2 = 0L) or (_col4 is not null or _col0 is null or (_col3 < _col2)) is not true) (type: boolean)
+                  Statistics: Num rows: 6 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: int)
                     outputColumnNames: _col0
-                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
-                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 24 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 
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), true (type: boolean)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Dummy Store
-            Execution mode: llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                Merge Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col1, _col2
-                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col2 (type: int)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col2 (type: int)
-                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
-        Reducer 7 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -773,6 +724,25 @@
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint), _col2 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: true (type: boolean), _col0 (type: int)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: int)
+                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator
@@ -836,10 +806,10 @@
     Tez
 #### A masked pattern was here ####
       Edges:
-        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
-        Reducer 7 <- Map 5 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+        Reducer 5 <- Map 4 (SIMPLE_EDGE)
+        Reducer 6 <- Map 4 (SIMPLE_EDGE)
 #### A masked pattern was here ####
       Vertices:
         Map 1 
@@ -857,24 +827,9 @@
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Filter Operator
-                    predicate: c1 is not null (type: boolean)
-                    Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
-                      keys: c1 (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 4 Data size: 16 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: 16 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
-        Map 5 
+        Map 4 
             Map Operator Tree:
                 TableScan
                   alias: t2
@@ -884,18 +839,6 @@
                     predicate: c1 is not null (type: boolean)
                     Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      keys: c1 (type: int)
-                      minReductionHashAggr: 0.4
-                      mode: hash
-                      outputColumnNames: _col0
-                      Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                    Group By Operator
                       aggregations: count(), count(c1)
                       keys: c1 (type: int)
                       minReductionHashAggr: 0.4
@@ -909,6 +852,18 @@
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col1 (type: bigint), _col2 (type: bigint)
+                    Group By Operator
+                      keys: c1 (type: int)
+                      minReductionHashAggr: 0.4
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 2 Data size: 8 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: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
@@ -937,58 +892,24 @@
                      Left Outer Join 0 to 1
                 keys:
                   0 _col0 (type: int)
-                  1 _col2 (type: int)
-                outputColumnNames: _col0, _col2, _col3, _col5
-                Statistics: Num rows: 9 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  1 _col1 (type: int)
+                outputColumnNames: _col0, _col2, _col3, _col4
+                Statistics: Num rows: 6 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                 Filter Operator
-                  predicate: (_col2 is null or (_col2 = 0L) or (_col5 is not null or _col0 is null or (_col3 < _col2)) is not true) (type: boolean)
-                  Statistics: Num rows: 9 Data size: 160 Basic stats: COMPLETE Column stats: COMPLETE
+                  predicate: (_col2 is null or (_col2 = 0L) or (_col4 is not null or _col0 is null or (_col3 < _col2)) is not true) (type: boolean)
+                  Statistics: Num rows: 6 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
                     expressions: _col0 (type: int)
                     outputColumnNames: _col0
-                    Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
                       compressed: false
-                      Statistics: Num rows: 9 Data size: 36 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 6 Data size: 24 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 
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-                Select Operator
-                  expressions: _col0 (type: int), true (type: boolean)
-                  outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Dummy Store
-            Execution mode: llap
-            Reduce Operator Tree:
-              Group By Operator
-                keys: KEY._col0 (type: int)
-                mode: mergepartial
-                outputColumnNames: _col0
-                Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                Merge Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col1, _col2
-                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                  Reduce Output Operator
-                    key expressions: _col2 (type: int)
-                    null sort order: z
-                    sort order: +
-                    Map-reduce partition columns: _col2 (type: int)
-                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
-                    value expressions: _col1 (type: boolean)
-        Reducer 7 
+        Reducer 5 
             Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
@@ -1004,6 +925,25 @@
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 2 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint), _col2 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: true (type: boolean), _col0 (type: int)
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    key expressions: _col1 (type: int)
+                    null sort order: z
+                    sort order: +
+                    Map-reduce partition columns: _col1 (type: int)
+                    Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: boolean)
 
   Stage: Stage-0
     Fetch Operator