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

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


The following commit(s) were added to refs/heads/master by this push:
     new b17ecf8e99c HIVE-27138: MapJoinOperator throws NPE when computing 
OuterJoin with filter expressions on small table (Seonggon Namgung, reviewed by 
Krisztian Kasa)
b17ecf8e99c is described below

commit b17ecf8e99c293ed250abc80b9a8ff95d37a61db
Author: seonggon <ln...@postech.ac.kr>
AuthorDate: Fri Sep 15 18:24:25 2023 +0900

    HIVE-27138: MapJoinOperator throws NPE when computing OuterJoin with filter 
expressions on small table (Seonggon Namgung, reviewed by Krisztian Kasa)
---
 .../persistence/MapJoinBytesTableContainer.java    |   2 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java      |   4 +-
 .../hive/ql/optimizer/FiltertagAppenderProc.java   | 229 +++++
 .../ql/optimizer/FullOuterMapJoinOptimization.java |  95 ---
 .../apache/hadoop/hive/ql/parse/TezCompiler.java   |  15 +
 .../ql/udf/generic/GenericUDFBaseArithmetic.java   |   2 +-
 .../hive/ql/udf/generic/GenericUDFBaseNumeric.java |   2 +-
 .../mapjoin_filter_on_outerjoin_tez.q              |  79 ++
 .../llap/fullouter_mapjoin_1_optimized.q.out       |   8 +-
 .../llap/mapjoin_filter_on_outerjoin_tez.q.out     | 948 +++++++++++++++++++++
 .../llap/vector_fullouter_mapjoin_1_fast.q.out     |  37 +-
 .../vector_fullouter_mapjoin_1_optimized.q.out     |  37 +-
 ...or_fullouter_mapjoin_1_optimized_passthru.q.out |  37 +-
 .../llap/vector_outer_join_constants.q.out         |   5 +-
 .../perf/tpcds30tb/tez/query97.q.out               |   1 +
 .../hive/serde2/lazybinary/LazyBinaryStruct.java   |   5 +-
 16 files changed, 1373 insertions(+), 133 deletions(-)

diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
index 464edf60f5d..79695975ef2 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
@@ -342,7 +342,7 @@ public class MapJoinBytesTableContainer
 
     @Override
     public byte updateStateByte(Byte previousValue) {
-      if (!hasTag || filterGetter == null) {
+      if (filterGetter == null) {
         return (byte) 0xff;
       }
       byte aliasFilter = (previousValue == null) ? (byte)0xff : 
previousValue.byteValue();
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index d68a1e2d436..57299f95699 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -1542,9 +1542,7 @@ public class ConvertJoinMapJoin implements 
SemanticNodeProcessor {
         LOG.info("Selected dynamic partitioned hash join");
         MapJoinDesc mapJoinDesc = mapJoinOp.getConf();
         mapJoinDesc.setDynamicPartitionHashJoin(true);
-        if (mapJoinConversion.getIsFullOuterJoin()) {
-          FullOuterMapJoinOptimization.removeFilterMap(mapJoinDesc);
-        }
+
         // Set OpTraits for dynamically partitioned hash join:
         // bucketColNames: Re-use previous joinOp's bucketColNames. Parent 
operators should be
         //   reduce sink, which should have bucket columns based on the join 
keys.
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FiltertagAppenderProc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FiltertagAppenderProc.java
new file mode 100644
index 00000000000..59295ff6bfa
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FiltertagAppenderProc.java
@@ -0,0 +1,229 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+import org.apache.hadoop.hive.ql.exec.ColumnInfo;
+import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
+import org.apache.hadoop.hive.ql.plan.PlanUtils;
+import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.apache.hadoop.hive.ql.udf.UDFToShort;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPMultiply;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNot;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPPlus;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+
+/**
+ * Append a filterTag computation column to ReduceSinkOperators whose child is 
MapJoinOperator.
+ * The added column expresses JoinUtil#isFiltered in the form of ExprNode.
+ * The added column should be located at the end of row as CommonJoinOperator 
expects it.
+ *
+ * This processor only affects small tables of MapJoin that run on Tez engine.
+ * For big table, MapJoinOperator#process() calls 
CommonJoinOperator#getFilteredValue(), which adds filterTag.
+ * For MapReduce engine, HashTableSinkOperator adds filterTag to every row.
+ */
+public class FiltertagAppenderProc implements SemanticNodeProcessor {
+
+  private final TypeInfo shortType = 
TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.SMALLINT_TYPE_NAME);
+
+  @Override
+  public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, 
Object... nodeOutputs)
+      throws SemanticException {
+    MapJoinOperator mapJoinOp = (MapJoinOperator) nd;
+    MapJoinDesc mapJoinDesc = mapJoinOp.getConf();
+
+    if (mapJoinDesc.getFilterMap() == null) {
+      return null;
+    }
+
+    int[][] filterMap = mapJoinDesc.getFilterMap();
+
+    // 1. Extend ReduceSinkoperator if it's output is filtered by 
MapJoinOperator.
+    for (byte pos = 0; pos < filterMap.length; pos++) {
+      if (pos == mapJoinDesc.getPosBigTable() || filterMap[pos] == null) {
+        continue;
+      }
+
+      ExprNodeDesc filterTagExpr =
+          generateFilterTagExpression(filterMap[pos], 
mapJoinDesc.getFilters().get(pos));
+
+      // Note that the parent RS for the given pos is retrieved in different 
way in MapJoinProcessor.
+      // TODO: MapJoinProcessor.convertMapJoin() fixes the order of parent 
operators.
+      //  Does other callers also fix the order as well as 
MapJoinProcessor.convertMapJoin()?
+      ReduceSinkOperator parent = (ReduceSinkOperator) 
mapJoinOp.getParentOperators().get(pos);
+      ReduceSinkDesc pRsConf = parent.getConf();
+
+      // MapJoinProcessor.getMapJoinDesc() replaces filter expressions with 
backtracked one if
+      // adjustParentsChildren is true.
+      // As of now, ConvertJoinMapJoin.convertJoinDynamicPartitionedHashJoin() 
is the only functions that
+      // calls this method with adjustParentsChildren = false. Therefore, we 
backtrack filter expressions
+      // only if MapJoinDesc.isDynamicPartitionHashJoin is true, which is also 
the unique property of
+      // ConvertJoinMapJoin.convertJoinDynamicPartitionedHashJoin().
+      ExprNodeDesc mapSideFilterTagExpr;
+      if (mapJoinDesc.isDynamicPartitionHashJoin()) {
+        mapSideFilterTagExpr = ExprNodeDescUtils.backtrack(filterTagExpr, 
mapJoinOp, parent);
+      } else {
+        mapSideFilterTagExpr = filterTagExpr;
+      }
+      String filterColumnName = "_filterTag";
+
+      pRsConf.getValueCols().add(mapSideFilterTagExpr);
+      pRsConf.getOutputValueColumnNames().add(filterColumnName);
+      pRsConf.getColumnExprMap()
+          .put(Utilities.ReduceField.VALUE + "." + filterColumnName, 
mapSideFilterTagExpr);
+
+      ColumnInfo filterTagColumnInfo =
+          new ColumnInfo(Utilities.ReduceField.VALUE + "." + filterColumnName, 
shortType, "", false);
+      parent.getSchema().getSignature().add(filterTagColumnInfo);
+
+      TableDesc newTableDesc =
+          PlanUtils.getReduceValueTableDesc(
+              PlanUtils.getFieldSchemasFromColumnList(pRsConf.getValueCols(), 
"_col"));
+      pRsConf.setValueSerializeInfo(newTableDesc);
+    }
+
+    // 2. Update MapJoinOperator's valueFilteredTableDescs.
+    // Unlike HashTableSinkOperator used in MR engine, Tez engine directly 
passes rows from RS to MapJoin.
+    // Therefore, RS's writer and MapJoin's reader should have the same 
TableDesc. We create valueTableDesc
+    // here again because it can be different from RS's valueSerializeInfo due 
to ColumnPruner.
+    List<TableDesc> newMapJoinValueFilteredTableDescs =
+        new ArrayList<>(mapJoinOp.getParentOperators().size());
+    for (byte pos = 0; pos < mapJoinOp.getParentOperators().size(); pos++) {
+      TableDesc tableDesc;
+
+      if (pos == mapJoinDesc.getPosBigTable() || filterMap[pos] == null) {
+        // We did not change corresponding parent operator. Use the original 
tableDesc.
+        tableDesc = mapJoinDesc.getValueFilteredTblDescs().get(pos);
+      } else {
+        // Create a new TableDesc based on corresponding parent RSOperator.
+        ReduceSinkOperator parent = (ReduceSinkOperator) 
mapJoinOp.getParentOperators().get(pos);
+        ReduceSinkDesc pRsConf = parent.getConf();
+
+        tableDesc =
+            PlanUtils.getMapJoinValueTableDesc(
+                
PlanUtils.getFieldSchemasFromColumnList(pRsConf.getValueCols(), 
"mapjoinvalue"));
+      }
+
+      newMapJoinValueFilteredTableDescs.add(tableDesc);
+    }
+    mapJoinDesc.setValueFilteredTblDescs(newMapJoinValueFilteredTableDescs);
+
+    return null;
+  }
+
+  /**
+   * Generate an ExprNodeDesc that expresses the following method:
+   * JoinUtil#isFiltered(Object, List<ExprNodeEvaluator>, 
List<ObjectInspector>, int[]).
+   */
+  private ExprNodeDesc generateFilterTagExpression(int[] filterMap, 
List<ExprNodeDesc> filterExprs) {
+    ExprNodeDesc filterTagExpr = new ExprNodeConstantDesc(shortType, (short) 
0);
+    Map<Byte, ExprNodeDesc> filterExprMap = getFilterExprMap(filterMap, 
filterExprs);
+
+    for (Map.Entry<Byte, ExprNodeDesc> entry: filterExprMap.entrySet()) {
+      ExprNodeDesc filterTagMaskExpr = generateFilterTagMask(entry.getKey(), 
entry.getValue());
+
+      if (filterTagExpr instanceof ExprNodeConstantDesc) {
+        filterTagExpr = filterTagMaskExpr;
+      } else {
+        List<ExprNodeDesc> plusArgs = Arrays.asList(filterTagMaskExpr, 
filterTagExpr);
+        filterTagExpr = new ExprNodeGenericFuncDesc(shortType, new 
GenericUDFOPPlus(), plusArgs);
+      }
+    }
+
+    return filterTagExpr;
+  }
+
+  /**
+   * Group filterExprs by tag and merge each of them into a single boolean 
ExprNodeDesc using AND operator.
+   * filterInfo is repetition of tag and the length of corresponding filter 
expressions.
+   * For example, filterInfo = {0, 2, 1, 3} means that the first 2 elements in 
filterExprs belong to tag 0,
+   * and the remaining 3 elements belong to tag 1.
+   */
+  private Map<Byte, ExprNodeDesc> getFilterExprMap(int[] filterInfo, 
List<ExprNodeDesc> filterExprs) {
+    Map<Byte, ExprNodeDesc> filterExprMap = new HashMap<>();
+
+    int exprListOffset = 0;
+    for (int idx = 0; idx < filterInfo.length; idx = idx + 2) {
+      byte tag = (byte) filterInfo[idx];
+      int length = filterInfo[idx + 1];
+
+      int nextExprOffset = exprListOffset + length;
+      List<ExprNodeDesc> andArgs = filterExprs.subList(exprListOffset, 
nextExprOffset);
+      exprListOffset = nextExprOffset;
+
+      if (andArgs.size() == 1) {
+        filterExprMap.put(tag, andArgs.get(0));
+      } else if (andArgs.size() > 1) {
+        filterExprMap.put(tag, ExprNodeDescUtils.and(andArgs));
+      }
+    }
+
+    return filterExprMap;
+  }
+
+  /**
+   * Generate an ExprNodeDesc that expresses the following code:
+   *   UDFToShort(!condition) * (short) (1 << tag),
+   * which is logically equivalent to
+   *   if (condition) { return (short) 0 } else { return (short) (1 << tag); }.
+   */
+  private ExprNodeDesc generateFilterTagMask(byte tag, ExprNodeDesc condition) 
{
+    ExprNodeDesc filterMaskValue = new ExprNodeConstantDesc(shortType, (short) 
(1 << tag));
+
+    List<ExprNodeDesc> negateArg = Collections.singletonList(condition);
+    ExprNodeDesc negate = new ExprNodeGenericFuncDesc(
+        TypeInfoFactory.getPrimitiveTypeInfo(serdeConstants.BOOLEAN_TYPE_NAME),
+        new GenericUDFOPNot(),
+        negateArg);
+
+    GenericUDFBridge toShort = new GenericUDFBridge();
+    toShort.setUdfClassName(UDFToShort.class.getName());
+    toShort.setUdfName(UDFToShort.class.getSimpleName());
+
+    List<ExprNodeDesc> toShortArg = Collections.singletonList(negate);
+    ExprNodeDesc conditionAsShort = new ExprNodeGenericFuncDesc(shortType, 
toShort, toShortArg);
+
+    List<ExprNodeDesc> multiplyArgs = Arrays.asList(conditionAsShort, 
filterMaskValue);
+    ExprNodeDesc multiply = new ExprNodeGenericFuncDesc(shortType, new 
GenericUDFOPMultiply(), multiplyArgs);
+
+    return multiply;
+  }
+}
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FullOuterMapJoinOptimization.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FullOuterMapJoinOptimization.java
deleted file mode 100644
index b9e86ebbc6a..00000000000
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/FullOuterMapJoinOptimization.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.optimizer;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Properties;
-import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
-import org.apache.hadoop.hive.ql.plan.TableDesc;
-import org.apache.hadoop.hive.serde.serdeConstants;
-import org.apache.hadoop.hive.serde2.SerDeUtils;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-
-import com.google.common.base.Joiner;
-
-/**
- * FULL OUTER MapJoin planning.
- */
-public class FullOuterMapJoinOptimization {
-
-  FullOuterMapJoinOptimization() {
-  }
-
-  public static void removeFilterMap(MapJoinDesc mapJoinDesc) throws 
SemanticException {
-    int[][] filterMaps = mapJoinDesc.getFilterMap();
-    if (filterMaps == null) {
-      return;
-    }
-    final byte posBigTable = (byte) mapJoinDesc.getPosBigTable();
-    final int numAliases = mapJoinDesc.getExprs().size();
-    List<TableDesc> valueFilteredTblDescs = 
mapJoinDesc.getValueFilteredTblDescs();
-    for (byte pos = 0; pos < numAliases; pos++) {
-      if (pos != posBigTable) {
-        int[] filterMap = filterMaps[pos];
-        TableDesc tableDesc = valueFilteredTblDescs.get(pos);
-        Properties properties = tableDesc.getProperties();
-        String columnNameProperty = 
properties.getProperty(serdeConstants.LIST_COLUMNS);
-        String columnNameDelimiter =
-            properties.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ?
-                properties.getProperty(serdeConstants.COLUMN_NAME_DELIMITER) :
-                  String.valueOf(SerDeUtils.COMMA);
-
-        String columnTypeProperty = 
properties.getProperty(serdeConstants.LIST_COLUMN_TYPES);
-        List<String> columnNameList;
-        if (columnNameProperty.length() == 0) {
-          columnNameList = new ArrayList<String>();
-        } else {
-          columnNameList = 
Arrays.asList(columnNameProperty.split(columnNameDelimiter));
-        }
-        List<String> truncatedColumnNameList = columnNameList.subList(0, 
columnNameList.size() - 1);
-        String truncatedColumnNameProperty =
-            Joiner.on(columnNameDelimiter).join(truncatedColumnNameList);
-
-        List<TypeInfo> columnTypeList;
-        if (columnTypeProperty.length() == 0) {
-          columnTypeList = new ArrayList<TypeInfo>();
-        } else {
-          columnTypeList = TypeInfoUtils
-              .getTypeInfosFromTypeString(columnTypeProperty);
-        }
-        if (!columnTypeList.get(columnTypeList.size() - 
1).equals(TypeInfoFactory.shortTypeInfo)) {
-          throw new SemanticException("Expecting filterTag smallint as last 
column type");
-        }
-        List<TypeInfo> truncatedColumnTypeList =
-            columnTypeList.subList(0, columnTypeList.size() - 1);
-        String truncatedColumnTypeProperty =
-            Joiner.on(",").join(truncatedColumnTypeList);
-
-        properties.setProperty(serdeConstants.LIST_COLUMNS, 
truncatedColumnNameProperty);
-        properties.setProperty(serdeConstants.LIST_COLUMN_TYPES, 
truncatedColumnTypeProperty);
-      }
-    }
-    mapJoinDesc.setFilterMap(null);
-  }
-}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
index 175747bc9e0..57f1f491b31 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
@@ -88,6 +88,7 @@ import org.apache.hadoop.hive.ql.optimizer.ConstantPropagate;
 import 
org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcCtx.ConstantPropagateOption;
 import org.apache.hadoop.hive.ql.optimizer.ConvertJoinMapJoin;
 import org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruningOptimization;
+import org.apache.hadoop.hive.ql.optimizer.FiltertagAppenderProc;
 import org.apache.hadoop.hive.ql.optimizer.MergeJoinProc;
 import org.apache.hadoop.hive.ql.optimizer.NonBlockingOpDeDupProc;
 import org.apache.hadoop.hive.ql.optimizer.ParallelEdgeFixer;
@@ -246,6 +247,8 @@ public class TezCompiler extends TaskCompiler {
     }
     perfLogger.perfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, 
"Shared scans optimization");
 
+    extendParentReduceSinkOfMapJoin(procCtx);
+
     // need a new run of the constant folding because we might have created 
lots
     // of "and true and true" conditions.
     // Rather than run the full constant folding just need to shortcut AND/OR 
expressions
@@ -493,6 +496,18 @@ public class TezCompiler extends TaskCompiler {
     ogw.startWalking(topNodes, null);
   }
 
+  private void extendParentReduceSinkOfMapJoin(OptimizeTezProcContext procCtx) 
throws SemanticException {
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<>();
+    opRules.put(
+        new RuleRegExp("Extend parent RS of MapJoin", 
MapJoinOperator.getOperatorName() + "%"),
+        new FiltertagAppenderProc());
+
+    SemanticDispatcher disp = new DefaultRuleDispatcher(null, opRules, 
procCtx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
+    List<Node> topNodes = new 
ArrayList<>(procCtx.parseContext.getTopOps().values());
+    ogw.startWalking(topNodes, null);
+  }
+
   private void semijoinRemovalBasedTransformations(OptimizeTezProcContext 
procCtx) throws SemanticException {
     PerfLogger perfLogger = SessionState.getPerfLogger();
 
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseArithmetic.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseArithmetic.java
index 0735950a752..d51f3a0b09a 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseArithmetic.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseArithmetic.java
@@ -50,7 +50,7 @@ public abstract class GenericUDFBaseArithmetic extends 
GenericUDFBaseBinary {
 
     // Lookup values needed for numeric arithmetic UDFs
     if (confLookupNeeded) {
-      CompatLevel compatLevel = 
HiveCompat.getCompatLevel(SessionState.get().getConf());
+      CompatLevel compatLevel = 
HiveCompat.getCompatLevel(SessionState.getSessionConf());
       ansiSqlArithmetic = compatLevel.ordinal() > 
CompatLevel.HIVE_0_12.ordinal();
       confLookupNeeded = false;
     }
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseNumeric.java 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseNumeric.java
index 34a03ed95d0..ed1a6f961f7 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseNumeric.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBaseNumeric.java
@@ -111,7 +111,7 @@ public abstract class GenericUDFBaseNumeric extends 
GenericUDFBaseBinary impleme
     // (where there should be valid HiveConf from SessionState).  Plan 
serialization will ensure
     // we have access to these values in the map/reduce tasks.
     if (confLookupNeeded) {
-      CompatLevel compatLevel = 
HiveCompat.getCompatLevel(SessionState.get().getConf());
+      CompatLevel compatLevel = 
HiveCompat.getCompatLevel(SessionState.getSessionConf());
       ansiSqlArithmetic = compatLevel.ordinal() > 
CompatLevel.HIVE_0_12.ordinal();
       confLookupNeeded = false;
     }
diff --git 
a/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin_tez.q 
b/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin_tez.q
new file mode 100644
index 00000000000..368fd774a0e
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin_tez.q
@@ -0,0 +1,79 @@
+--! qt:dataset:src1
+--! qt:dataset:src
+set hive.auto.convert.join = false;
+set hive.merge.nway.joins=true;
+-- SORT_QUERY_RESULTS
+
+--HIVE-2101 mapjoin sometimes gives wrong results if there is a filter in the 
on condition
+
+SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key;
+
+explain
+SELECT /*+ mapjoin(src1, src2) */ * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key;
+
+SELECT /*+ mapjoin(src1, src2) */ * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key;
+
+set hive.auto.convert.join = true;
+
+explain
+SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key;
+
+SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key;
+
+set hive.optimize.shared.work=true;
+set hive.vectorized.execution.enabled=true;
+
+explain
+SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = src2.key AND 
src1.key < 10);
+SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = src2.key AND 
src1.key < 10);
+
+explain
+SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = src2.key AND 
src2.key < 10);
+SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = src2.key AND 
src2.key < 10);
+
+
+-- Test FullOuterJoin with filters on both tables
+
+set hive.optimize.dynamic.partition.hashjoin=true;
+DROP TABLE IF EXISTS c;
+CREATE TABLE c (key int, value int);
+INSERT INTO c VALUES (1, 0), (2, 0);
+DROP TABLE IF EXISTS d;
+CREATE TABLE d (key int, value int);
+INSERT INTO d VALUES (1, 1), (2, 1);
+
+-- TOOD: Currently VectorMapJoin returns wrong result.
+-- set hive.auto.convert.join=true;
+-- set hive.vectorized.execution.enabled=true;
+
+-- explain
+-- SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key 
> 1;
+-- SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key 
> 1;
+
+set hive.auto.convert.join=true;
+set hive.vectorized.execution.enabled=false;
+
+explain
+SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key > 1;
+SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key > 1;
+
+set hive.auto.convert.join=false;
+
+explain
+SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key > 1;
+SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key > 1;
diff --git 
a/ql/src/test/results/clientpositive/llap/fullouter_mapjoin_1_optimized.q.out 
b/ql/src/test/results/clientpositive/llap/fullouter_mapjoin_1_optimized.q.out
index ead94786689..8c9d22a2683 100644
--- 
a/ql/src/test/results/clientpositive/llap/fullouter_mapjoin_1_optimized.q.out
+++ 
b/ql/src/test/results/clientpositive/llap/fullouter_mapjoin_1_optimized.q.out
@@ -415,6 +415,7 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: _col0 (type: bigint)
                       Statistics: Num rows: 11 Data size: 124 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -894,6 +895,7 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: _col0 (type: smallint)
                       Statistics: Num rows: 13 Data size: 63 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -1169,7 +1171,7 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 11 Data size: 173 Basic stats: 
COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      value expressions: _col1 (type: string), 0S (type: 
smallint)
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -1453,6 +1455,7 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 12 Data size: 106 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -1813,6 +1816,7 @@ STAGE PLANS:
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: smallint), 
_col1 (type: int)
                       Statistics: Num rows: 13 Data size: 140 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -2468,6 +2472,7 @@ STAGE PLANS:
                       sort order: +++
                       Map-reduce partition columns: _col0 (type: timestamp), 
_col1 (type: smallint), _col2 (type: string)
                       Statistics: Num rows: 17 Data size: 1729 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
@@ -2912,6 +2917,7 @@ STAGE PLANS:
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 13 Data size: 1108 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: llap
             LLAP IO: all inputs
         Map 2 
diff --git 
a/ql/src/test/results/clientpositive/llap/mapjoin_filter_on_outerjoin_tez.q.out 
b/ql/src/test/results/clientpositive/llap/mapjoin_filter_on_outerjoin_tez.q.out
new file mode 100644
index 00000000000..5080aed0950
--- /dev/null
+++ 
b/ql/src/test/results/clientpositive/llap/mapjoin_filter_on_outerjoin_tez.q.out
@@ -0,0 +1,948 @@
+PREHOOK: query: SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+NULL   NULL    128             128     val_128
+NULL   NULL    128             128     val_128
+NULL   NULL    128             128     val_128
+NULL   NULL    146     val_146 146     val_146
+NULL   NULL    146     val_146 146     val_146
+NULL   NULL    150     val_150 150     val_150
+NULL   NULL    213     val_213 213     val_213
+NULL   NULL    213     val_213 213     val_213
+NULL   NULL    224             224     val_224
+NULL   NULL    224             224     val_224
+NULL   NULL    238     val_238 238     val_238
+NULL   NULL    238     val_238 238     val_238
+NULL   NULL    255     val_255 255     val_255
+NULL   NULL    255     val_255 255     val_255
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    278     val_278 278     val_278
+NULL   NULL    278     val_278 278     val_278
+NULL   NULL    66      val_66  66      val_66
+NULL   NULL    98      val_98  98      val_98
+NULL   NULL    98      val_98  98      val_98
+PREHOOK: query: explain
+SELECT /*+ mapjoin(src1, src2) */ * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT /*+ mapjoin(src1, src2) */ * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 
(SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src1
+                  filterExpr: (UDFToDouble(key) < 10.0D) (type: boolean)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+                    Statistics: Num rows: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 8 Data size: 1400 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: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  filterExpr: (UDFToDouble(key) < 300.0D) (type: boolean)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) < 300.0D) (type: boolean)
+                    Statistics: Num rows: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 8 Data size: 1400 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: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: src3
+                  filterExpr: (UDFToDouble(key) < 300.0D) (type: boolean)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) < 300.0D) (type: boolean)
+                    Statistics: Num rows: 166 Data size: 29548 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 166 Data size: 29548 Basic 
stats: COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Right Outer Join 0 to 1
+                     Inner Join 1 to 2
+                filter predicates:
+                  0 
+                  1 {(UDFToDouble(KEY.reducesinkkey0) > 10.0D)}
+                  2 
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                  2 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 8 Data size: 4224 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col0 (type: string), _col2 (type: string), 
_col4 (type: string)
+                  null sort order: zzz
+                  sort order: +++
+                  Statistics: Num rows: 8 Data size: 4224 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string), _col3 (type: 
string), _col5 (type: string)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 
(type: string), KEY.reducesinkkey1 (type: string), VALUE._col1 (type: string), 
KEY.reducesinkkey2 (type: string), VALUE._col2 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 8 Data size: 4224 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 8 Data size: 4224 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT /*+ mapjoin(src1, src2) */ * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT /*+ mapjoin(src1, src2) */ * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+NULL   NULL    128             128     val_128
+NULL   NULL    128             128     val_128
+NULL   NULL    128             128     val_128
+NULL   NULL    146     val_146 146     val_146
+NULL   NULL    146     val_146 146     val_146
+NULL   NULL    150     val_150 150     val_150
+NULL   NULL    213     val_213 213     val_213
+NULL   NULL    213     val_213 213     val_213
+NULL   NULL    224             224     val_224
+NULL   NULL    224             224     val_224
+NULL   NULL    238     val_238 238     val_238
+NULL   NULL    238     val_238 238     val_238
+NULL   NULL    255     val_255 255     val_255
+NULL   NULL    255     val_255 255     val_255
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    278     val_278 278     val_278
+NULL   NULL    278     val_278 278     val_278
+NULL   NULL    66      val_66  66      val_66
+NULL   NULL    98      val_98  98      val_98
+NULL   NULL    98      val_98  98      val_98
+PREHOOK: query: explain
+SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 3 <- Map 1 (BROADCAST_EDGE), Reducer 2 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+        Reducer 4 <- Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  filterExpr: ((UDFToDouble(key) < 300.0D) or 
(UDFToDouble(key) < 10.0D)) (type: boolean)
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) < 300.0D) (type: boolean)
+                    Statistics: Num rows: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 8 Data size: 1400 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: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+                  Filter Operator
+                    predicate: (UDFToDouble(key) < 10.0D) (type: boolean)
+                    Statistics: Num rows: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 8 Data size: 1400 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: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: src3
+                  filterExpr: (UDFToDouble(key) < 300.0D) (type: boolean)
+                  Statistics: Num rows: 500 Data size: 89000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (UDFToDouble(key) < 300.0D) (type: boolean)
+                    Statistics: Num rows: 166 Data size: 29548 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 29548 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Map Join Operator
+                        condition map:
+                             Right Outer Join 0 to 1
+                             Inner Join 1 to 2
+                        filter predicates:
+                          0 
+                          1 {(UDFToDouble(_col0) > 10.0D)}
+                          2 
+                        keys:
+                          0 _col0 (type: string)
+                          1 _col0 (type: string)
+                          2 _col0 (type: string)
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5
+                        input vertices:
+                          0 Map 1
+                          1 Reducer 2
+                        Statistics: Num rows: 8 Data size: 4224 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string), _col2 (type: 
string), _col4 (type: string)
+                          null sort order: zzz
+                          sort order: +++
+                          Statistics: Num rows: 8 Data size: 4224 Basic stats: 
COMPLETE Column stats: COMPLETE
+                          value expressions: _col1 (type: string), _col3 
(type: string), _col5 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 
(type: string)
+                outputColumnNames: _col0, _col1
+                Reduce Output Operator
+                  key expressions: _col0 (type: string)
+                  null sort order: z
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: string)
+                  Statistics: Num rows: 8 Data size: 1400 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  value expressions: _col1 (type: string), (UDFToShort((not 
(UDFToDouble(_col0) > 10.0D))) * 1S) (type: smallint)
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 
(type: string), KEY.reducesinkkey1 (type: string), VALUE._col1 (type: string), 
KEY.reducesinkkey2 (type: string), VALUE._col2 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 8 Data size: 4224 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 8 Data size: 4224 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM src1
+  RIGHT OUTER JOIN src1 src2 ON (src1.key = src2.key AND src1.key < 10 AND 
src2.key > 10)
+  JOIN src src3 ON (src2.key = src3.key AND src3.key < 300)
+  SORT BY src1.key, src2.key, src3.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+NULL   NULL    128             128     val_128
+NULL   NULL    128             128     val_128
+NULL   NULL    128             128     val_128
+NULL   NULL    146     val_146 146     val_146
+NULL   NULL    146     val_146 146     val_146
+NULL   NULL    150     val_150 150     val_150
+NULL   NULL    213     val_213 213     val_213
+NULL   NULL    213     val_213 213     val_213
+NULL   NULL    224             224     val_224
+NULL   NULL    224             224     val_224
+NULL   NULL    238     val_238 238     val_238
+NULL   NULL    238     val_238 238     val_238
+NULL   NULL    255     val_255 255     val_255
+NULL   NULL    255     val_255 255     val_255
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    273     val_273 273     val_273
+NULL   NULL    278     val_278 278     val_278
+NULL   NULL    278     val_278 278     val_278
+NULL   NULL    66      val_66  66      val_66
+NULL   NULL    98      val_98  98      val_98
+NULL   NULL    98      val_98  98      val_98
+PREHOOK: query: explain
+SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = src2.key AND 
src1.key < 10)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = src2.key AND 
src1.key < 10)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src1
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string), 
(UDFToDouble(key) < 10.0D) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 25 Data size: 4475 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: 25 Data size: 4475 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string), _col2 (type: 
boolean)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Full Outer Join 0 to 1
+                filter predicates:
+                  0 {VALUE._col1}
+                  1 
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col3, _col4
+                Statistics: Num rows: 50 Data size: 17500 Basic stats: 
COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: string), _col1 (type: string), 
_col3 (type: string), _col4 (type: string)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 50 Data size: 17500 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 50 Data size: 17500 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    table:
+                        input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: 
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = 
src2.key AND src1.key < 10)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = 
src2.key AND src1.key < 10)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+               NULL    NULL
+               NULL    NULL
+               NULL    NULL
+               NULL    NULL
+       val_165 NULL    NULL
+       val_193 NULL    NULL
+       val_265 NULL    NULL
+       val_27  NULL    NULL
+       val_409 NULL    NULL
+       val_484 NULL    NULL
+128            NULL    NULL
+146    val_146 NULL    NULL
+150    val_150 NULL    NULL
+213    val_213 NULL    NULL
+224            NULL    NULL
+238    val_238 NULL    NULL
+255    val_255 NULL    NULL
+273    val_273 NULL    NULL
+278    val_278 NULL    NULL
+311    val_311 NULL    NULL
+369            NULL    NULL
+401    val_401 NULL    NULL
+406    val_406 NULL    NULL
+66     val_66  NULL    NULL
+98     val_98  NULL    NULL
+NULL   NULL            
+NULL   NULL            
+NULL   NULL            
+NULL   NULL            
+NULL   NULL            val_165
+NULL   NULL            val_193
+NULL   NULL            val_265
+NULL   NULL            val_27
+NULL   NULL            val_409
+NULL   NULL            val_484
+NULL   NULL    128     
+NULL   NULL    146     val_146
+NULL   NULL    150     val_150
+NULL   NULL    213     val_213
+NULL   NULL    224     
+NULL   NULL    238     val_238
+NULL   NULL    255     val_255
+NULL   NULL    273     val_273
+NULL   NULL    278     val_278
+NULL   NULL    311     val_311
+NULL   NULL    369     
+NULL   NULL    401     val_401
+NULL   NULL    406     val_406
+NULL   NULL    66      val_66
+NULL   NULL    98      val_98
+PREHOOK: query: explain
+SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = src2.key AND 
src2.key < 10)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = src2.key AND 
src2.key < 10)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src1
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: string)
+                      Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: src2
+                  Statistics: Num rows: 25 Data size: 4375 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: string), value (type: string), 
(UDFToDouble(key) < 10.0D) (type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 25 Data size: 4475 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: 25 Data size: 4475 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string), _col2 (type: 
boolean)
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Full Outer Join 0 to 1
+                filter predicates:
+                  0 
+                  1 {VALUE._col1}
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 50 Data size: 17500 Basic stats: 
COMPLETE Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 50 Data size: 17500 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = 
src2.key AND src2.key < 10)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM src1 FULL OUTER JOIN src1 src2 ON (src1.key = 
src2.key AND src2.key < 10)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+               NULL    NULL
+               NULL    NULL
+               NULL    NULL
+               NULL    NULL
+       val_165 NULL    NULL
+       val_193 NULL    NULL
+       val_265 NULL    NULL
+       val_27  NULL    NULL
+       val_409 NULL    NULL
+       val_484 NULL    NULL
+128            NULL    NULL
+146    val_146 NULL    NULL
+150    val_150 NULL    NULL
+213    val_213 NULL    NULL
+224            NULL    NULL
+238    val_238 NULL    NULL
+255    val_255 NULL    NULL
+273    val_273 NULL    NULL
+278    val_278 NULL    NULL
+311    val_311 NULL    NULL
+369            NULL    NULL
+401    val_401 NULL    NULL
+406    val_406 NULL    NULL
+66     val_66  NULL    NULL
+98     val_98  NULL    NULL
+NULL   NULL            
+NULL   NULL            
+NULL   NULL            
+NULL   NULL            
+NULL   NULL            val_165
+NULL   NULL            val_193
+NULL   NULL            val_265
+NULL   NULL            val_27
+NULL   NULL            val_409
+NULL   NULL            val_484
+NULL   NULL    128     
+NULL   NULL    146     val_146
+NULL   NULL    150     val_150
+NULL   NULL    213     val_213
+NULL   NULL    224     
+NULL   NULL    238     val_238
+NULL   NULL    255     val_255
+NULL   NULL    273     val_273
+NULL   NULL    278     val_278
+NULL   NULL    311     val_311
+NULL   NULL    369     
+NULL   NULL    401     val_401
+NULL   NULL    406     val_406
+NULL   NULL    66      val_66
+NULL   NULL    98      val_98
+PREHOOK: query: DROP TABLE IF EXISTS c
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: DROP TABLE IF EXISTS c
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: CREATE TABLE c (key int, value int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@c
+POSTHOOK: query: CREATE TABLE c (key int, value int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@c
+PREHOOK: query: INSERT INTO c VALUES (1, 0), (2, 0)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@c
+POSTHOOK: query: INSERT INTO c VALUES (1, 0), (2, 0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@c
+POSTHOOK: Lineage: c.key SCRIPT []
+POSTHOOK: Lineage: c.value SCRIPT []
+PREHOOK: query: DROP TABLE IF EXISTS d
+PREHOOK: type: DROPTABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: DROP TABLE IF EXISTS d
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Output: database:default
+PREHOOK: query: CREATE TABLE d (key int, value int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@d
+POSTHOOK: query: CREATE TABLE d (key int, value int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@d
+PREHOOK: query: INSERT INTO d VALUES (1, 1), (2, 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@d
+POSTHOOK: query: INSERT INTO d VALUES (1, 1), (2, 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@d
+POSTHOOK: Lineage: d.key SCRIPT []
+POSTHOOK: Lineage: d.value SCRIPT []
+PREHOOK: query: explain
+SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key > 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c
+PREHOOK: Input: default@d
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key > 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c
+POSTHOOK: Input: default@d
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: int), (key > 0) 
(type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 2 Data size: 24 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: 24 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: int), _col2 (type: 
boolean)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: d
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: int), (key > 1) 
(type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 2 Data size: 24 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: 24 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: int), _col2 (type: 
boolean), (UDFToShort((not _col2)) * 1S) (type: smallint)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Full Outer Join 0 to 1
+                filter predicates:
+                  0 {VALUE._col1}
+                  1 {VALUE._col1}
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0, _col1, _col3, _col4
+                input vertices:
+                  1 Map 3
+                Statistics: Num rows: 4 Data size: 64 Basic stats: COMPLETE 
Column stats: COMPLETE
+                DynamicPartitionHashJoin: true
+                Select Operator
+                  expressions: _col0 (type: int), _col1 (type: int), _col3 
(type: int), _col4 (type: int)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 4 Data size: 64 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 4 Data size: 64 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    table:
+                        input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: 
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 
0 AND d.key > 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c
+PREHOOK: Input: default@d
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key 
> 0 AND d.key > 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c
+POSTHOOK: Input: default@d
+#### A masked pattern was here ####
+1      0       NULL    NULL
+2      0       2       1
+NULL   NULL    1       1
+PREHOOK: query: explain
+SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key > 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c
+PREHOOK: Input: default@d
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 0 AND d.key > 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c
+POSTHOOK: Input: default@d
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: int), (key > 0) 
(type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 2 Data size: 24 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: 24 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: int), _col2 (type: 
boolean)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: d
+                  Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: key (type: int), value (type: int), (key > 1) 
(type: boolean)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 2 Data size: 24 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: 24 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: int), _col2 (type: 
boolean)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Full Outer Join 0 to 1
+                filter predicates:
+                  0 {VALUE._col1}
+                  1 {VALUE._col1}
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1, _col3, _col4
+                Statistics: Num rows: 4 Data size: 64 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Select Operator
+                  expressions: _col0 (type: int), _col1 (type: int), _col3 
(type: int), _col4 (type: int)
+                  outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 4 Data size: 64 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 4 Data size: 64 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    table:
+                        input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: 
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key > 
0 AND d.key > 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@c
+PREHOOK: Input: default@d
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * from c FULL OUTER JOIN d on c.key = d.key AND c.key 
> 0 AND d.key > 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@c
+POSTHOOK: Input: default@d
+#### A masked pattern was here ####
+1      0       NULL    NULL
+2      0       2       1
+NULL   NULL    1       1
diff --git 
a/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_fast.q.out 
b/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_fast.q.out
index 33e1333a291..3cfdf6453cc 100644
--- 
a/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_fast.q.out
+++ 
b/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_fast.q.out
@@ -504,7 +504,10 @@ STAGE PLANS:
                           keyColumns: 0:bigint
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 11 Data size: 124 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -521,7 +524,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:bigint
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1087,7 +1090,10 @@ STAGE PLANS:
                           keyColumns: 0:smallint
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 13 Data size: 63 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1104,7 +1110,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:smallint
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1466,9 +1472,10 @@ STAGE PLANS:
                           keyColumns: 0:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                          valueColumns: 1:string
+                          valueColumns: 1:string, 4:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
4:smallint
                       Statistics: Num rows: 11 Data size: 173 Basic stats: 
COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      value expressions: _col1 (type: string), 0S (type: 
smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1485,7 +1492,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: key:int, b_string:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1856,7 +1863,10 @@ STAGE PLANS:
                           keyColumns: 0:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 12 Data size: 106 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1873,7 +1883,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:int
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -2319,7 +2329,10 @@ STAGE PLANS:
                           keyColumns: 0:smallint, 1:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 4:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
4:smallint
                       Statistics: Num rows: 13 Data size: 140 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -2336,7 +2349,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: key0:smallint, key1:int
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -3077,7 +3090,10 @@ STAGE PLANS:
                           keyColumns: 0:timestamp, 1:smallint, 2:string
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 5:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
5:smallint
                       Statistics: Num rows: 17 Data size: 1729 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -3094,7 +3110,7 @@ STAGE PLANS:
                     includeColumns: [0, 1, 2]
                     dataColumns: key0:timestamp, key1:smallint, key2:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -3625,7 +3641,10 @@ STAGE PLANS:
                           keyColumns: 0:string
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 13 Data size: 1108 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -3642,7 +3661,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
diff --git 
a/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_optimized.q.out
 
b/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_optimized.q.out
index 6c1a260a188..4b19c0c07f9 100644
--- 
a/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_optimized.q.out
+++ 
b/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_optimized.q.out
@@ -504,7 +504,10 @@ STAGE PLANS:
                           keyColumns: 0:bigint
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 11 Data size: 124 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -521,7 +524,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:bigint
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1087,7 +1090,10 @@ STAGE PLANS:
                           keyColumns: 0:smallint
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 13 Data size: 63 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1104,7 +1110,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:smallint
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1466,9 +1472,10 @@ STAGE PLANS:
                           keyColumns: 0:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                          valueColumns: 1:string
+                          valueColumns: 1:string, 4:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
4:smallint
                       Statistics: Num rows: 11 Data size: 173 Basic stats: 
COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      value expressions: _col1 (type: string), 0S (type: 
smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1485,7 +1492,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: key:int, b_string:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1856,7 +1863,10 @@ STAGE PLANS:
                           keyColumns: 0:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 12 Data size: 106 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1873,7 +1883,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:int
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -2319,7 +2329,10 @@ STAGE PLANS:
                           keyColumns: 0:smallint, 1:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 4:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
4:smallint
                       Statistics: Num rows: 13 Data size: 140 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -2336,7 +2349,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: key0:smallint, key1:int
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -3077,7 +3090,10 @@ STAGE PLANS:
                           keyColumns: 0:timestamp, 1:smallint, 2:string
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 5:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
5:smallint
                       Statistics: Num rows: 17 Data size: 1729 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -3094,7 +3110,7 @@ STAGE PLANS:
                     includeColumns: [0, 1, 2]
                     dataColumns: key0:timestamp, key1:smallint, key2:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -3625,7 +3641,10 @@ STAGE PLANS:
                           keyColumns: 0:string
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 13 Data size: 1108 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -3642,7 +3661,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
diff --git 
a/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_optimized_passthru.q.out
 
b/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_optimized_passthru.q.out
index 1d21674512f..93e7c693380 100644
--- 
a/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_optimized_passthru.q.out
+++ 
b/ql/src/test/results/clientpositive/llap/vector_fullouter_mapjoin_1_optimized_passthru.q.out
@@ -504,7 +504,10 @@ STAGE PLANS:
                           keyColumns: 0:bigint
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 11 Data size: 124 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -521,7 +524,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:bigint
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1084,7 +1087,10 @@ STAGE PLANS:
                           keyColumns: 0:smallint
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 13 Data size: 63 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1101,7 +1107,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:smallint
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1460,9 +1466,10 @@ STAGE PLANS:
                           keyColumns: 0:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                          valueColumns: 1:string
+                          valueColumns: 1:string, 4:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
4:smallint
                       Statistics: Num rows: 11 Data size: 173 Basic stats: 
COMPLETE Column stats: NONE
-                      value expressions: _col1 (type: string)
+                      value expressions: _col1 (type: string), 0S (type: 
smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1479,7 +1486,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: key:int, b_string:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -1846,7 +1853,10 @@ STAGE PLANS:
                           keyColumns: 0:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 12 Data size: 106 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -1863,7 +1873,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:int
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -2306,7 +2316,10 @@ STAGE PLANS:
                           keyColumns: 0:smallint, 1:int
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 4:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
4:smallint
                       Statistics: Num rows: 13 Data size: 140 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -2323,7 +2336,7 @@ STAGE PLANS:
                     includeColumns: [0, 1]
                     dataColumns: key0:smallint, key1:int
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -3061,7 +3074,10 @@ STAGE PLANS:
                           keyColumns: 0:timestamp, 1:smallint, 2:string
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 5:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
5:smallint
                       Statistics: Num rows: 17 Data size: 1729 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -3078,7 +3094,7 @@ STAGE PLANS:
                     includeColumns: [0, 1, 2]
                     dataColumns: key0:timestamp, key1:smallint, key2:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
@@ -3606,7 +3622,10 @@ STAGE PLANS:
                           keyColumns: 0:string
                           native: true
                           nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 3:smallint
+                          valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                       Statistics: Num rows: 13 Data size: 1108 Basic stats: 
COMPLETE Column stats: NONE
+                      value expressions: 0S (type: smallint)
             Execution mode: vectorized, llap
             LLAP IO: all inputs
             Map Vectorization:
@@ -3623,7 +3642,7 @@ STAGE PLANS:
                     includeColumns: [0]
                     dataColumns: key:string
                     partitionColumnCount: 0
-                    scratchColumnTypeNames: []
+                    scratchColumnTypeNames: [bigint]
         Map 2 
             Map Operator Tree:
                 TableScan
diff --git 
a/ql/src/test/results/clientpositive/llap/vector_outer_join_constants.q.out 
b/ql/src/test/results/clientpositive/llap/vector_outer_join_constants.q.out
index 93b8e91d5d1..8ea8348431b 100644
--- a/ql/src/test/results/clientpositive/llap/vector_outer_join_constants.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_outer_join_constants.q.out
@@ -653,9 +653,10 @@ STAGE PLANS:
                         className: VectorReduceSinkEmptyKeyOperator
                         native: true
                         nativeConditionsMet: 
hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine 
tez IN [tez] IS true, No PTF TopN IS true, No DISTINCT columns IS true, 
BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
-                        valueColumns: 1:int, 2:timestamp
+                        valueColumns: 1:int, 2:timestamp, 3:smallint
+                        valueExpressions: ConstantVectorExpression(val 0) -> 
3:smallint
                     Statistics: Num rows: 1 Data size: 44 Basic stats: 
COMPLETE Column stats: COMPLETE
-                    value expressions: _col0 (type: int), _col1 (type: 
timestamp)
+                    value expressions: _col0 (type: int), _col1 (type: 
timestamp), 0S (type: smallint)
         Reducer 3 
             Execution mode: vectorized, llap
             Reduce Vectorization:
diff --git 
a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query97.q.out 
b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query97.q.out
index cb0b2e93e10..3d01be7c28f 100644
--- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query97.q.out
+++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query97.q.out
@@ -214,6 +214,7 @@ STAGE PLANS:
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: bigint), _col1 
(type: bigint)
                   Statistics: Num rows: 8395118768 Data size: 133476173240 
Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: 0S (type: smallint)
 
   Stage: Stage-0
     Fetch Operator
diff --git 
a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryStruct.java 
b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryStruct.java
index 02aa14a5b61..af34b4c8f25 100644
--- 
a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryStruct.java
+++ 
b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryStruct.java
@@ -210,7 +210,8 @@ public class LazyBinaryStruct extends 
LazyBinaryNonPrimitive<LazyBinaryStructObj
     private final int fieldIndex;
     private final RecordInfo recordInfo = new LazyBinaryUtils.RecordInfo();
     private byte[] fieldBytes;
-    private int fieldStart, fieldLength;
+    private int fieldStart;
+    private int fieldLength;
     public SingleFieldGetter(LazyBinaryStructObjectInspector soi, int 
fieldIndex) {
       this.soi = soi;
       this.fieldIndex = fieldIndex;
@@ -221,7 +222,7 @@ public class LazyBinaryStruct extends 
LazyBinaryNonPrimitive<LazyBinaryStructObj
       fieldBytes = src.getBytes();
       int length = src.getLength();
       byte nullByte = fieldBytes[0];
-      int lastFieldByteEnd = 1, fieldStart = -1, fieldLength = -1;
+      int lastFieldByteEnd = 1;
       for (int i = 0; i <= fieldIndex; i++) {
         if ((nullByte & (1 << (i % 8))) != 0) {
           LazyBinaryUtils.checkObjectByteInfo(fieldRefs.get(i)

Reply via email to