HIVE-17225: HoS DPP pruning sink ops can target parallel work objects (Sahil 
Takiar, reviewed by Sergio Pena)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/90e68288
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/90e68288
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/90e68288

Branch: refs/heads/hive-14535
Commit: 90e682887de28d0c4c5a0caeac8f15d8a8b671af
Parents: e3c71b0
Author: Sahil Takiar <takiar.sa...@gmail.com>
Authored: Mon Sep 4 15:33:51 2017 -0700
Committer: Sahil Takiar <stak...@cloudera.com>
Committed: Mon Sep 4 15:33:51 2017 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |    1 +
 .../hadoop/hive/ql/exec/OperatorUtils.java      |   31 +
 .../SparkDynamicPartitionPruningResolver.java   |  146 +++
 .../physical/SparkMapJoinResolver.java          |   35 +-
 .../spark/SparkPartitionPruningSinkDesc.java    |   13 +-
 .../spark/SparkSkewJoinProcFactory.java         |    6 +-
 .../hive/ql/parse/spark/GenSparkUtils.java      |    2 +-
 .../hive/ql/parse/spark/SparkCompiler.java      |    5 +
 ...ynamic_partition_pruning_recursive_mapjoin.q |  147 +++
 ...spark_vectorized_dynamic_partition_pruning.q |    8 +-
 ...ic_partition_pruning_recursive_mapjoin.q.out | 1097 ++++++++++++++++++
 ...k_vectorized_dynamic_partition_pruning.q.out |  158 +--
 12 files changed, 1540 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties 
b/itests/src/test/resources/testconfiguration.properties
index 7385df6..663c95b 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1408,6 +1408,7 @@ 
miniSparkOnYarn.only.query.files=spark_combine_equivalent_work.q,\
   spark_dynamic_partition_pruning_3.q,\
   spark_dynamic_partition_pruning_mapjoin_only.q,\
   spark_constprog_dpp.q,\
+  spark_dynamic_partition_pruning_recursive_mapjoin.q,\
   dynamic_rdd_cache.q, \
   spark_multi_insert_parallel_orderby.q,\
   spark_explainuser_1.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
index 7308b5f..e79d100 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
@@ -24,10 +24,13 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.exec.NodeUtils.Function;
 import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
+import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
+import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.slf4j.Logger;
@@ -424,4 +427,32 @@ public class OperatorUtils {
     }
     return true;
   }
+
+  public static Set<Operator<?>> getOp(BaseWork work, Class<?> clazz) {
+    Set<Operator<?>> ops = new HashSet<Operator<?>>();
+    if (work instanceof MapWork) {
+      Collection<Operator<?>> opSet = ((MapWork) 
work).getAliasToWork().values();
+      Stack<Operator<?>> opStack = new Stack<Operator<?>>();
+      opStack.addAll(opSet);
+
+      while (!opStack.empty()) {
+        Operator<?> op = opStack.pop();
+        ops.add(op);
+        if (op.getChildOperators() != null) {
+          opStack.addAll(op.getChildOperators());
+        }
+      }
+    } else {
+      ops.addAll(work.getAllOperators());
+    }
+
+    Set<Operator<? extends OperatorDesc>> matchingOps =
+      new HashSet<Operator<? extends OperatorDesc>>();
+    for (Operator<? extends OperatorDesc> op : ops) {
+      if (clazz.isInstance(op)) {
+        matchingOps.add(op);
+      }
+    }
+    return matchingOps;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java
new file mode 100644
index 0000000..4d5c234
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java
@@ -0,0 +1,146 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.physical;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.Stack;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
+import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities;
+import org.apache.hadoop.hive.ql.lib.Dispatcher;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
+import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
+import org.apache.hadoop.hive.ql.plan.BaseWork;
+import org.apache.hadoop.hive.ql.plan.MapWork;
+
+
+/**
+ * A physical optimization that disables DPP if the source {@link MapWork} and 
target {@link MapWork} aren't in
+ * dependent {@link SparkTask}s.
+ *
+ * <p>
+ *   When DPP is run, the source {@link MapWork} produces a temp file that is 
read by the target {@link MapWork}. The
+ *   source {@link MapWork} must be run before the target {@link MapWork} is 
run, otherwise the target {@link MapWork}
+ *   will throw a {@link java.io.FileNotFoundException}. In order to guarantee 
this, the source {@link MapWork} must be
+ *   inside a {@link SparkTask} that runs before the {@link SparkTask} 
containing the target {@link MapWork}.
+ * </p>
+ *
+ * <p>
+ *   This {@link PhysicalPlanResolver} works by walking through the {@link 
Task} DAG and iterating over all the
+ *   {@link SparkPartitionPruningSinkOperator}s inside the {@link SparkTask}. 
For each sink operator, it takes the
+ *   target {@link MapWork} and checks if it exists in any of the child {@link 
SparkTask}s. If the target {@link MapWork}
+ *   is not in any child {@link SparkTask} then it removes the operator 
subtree that contains the
+ *   {@link SparkPartitionPruningSinkOperator}.
+ * </p>
+ */
+public class SparkDynamicPartitionPruningResolver implements 
PhysicalPlanResolver {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkDynamicPartitionPruningResolver.class.getName());
+
+  @Override
+  public PhysicalContext resolve(PhysicalContext pctx) throws 
SemanticException {
+    // Walk through the Task Graph and invoke 
SparkDynamicPartitionPruningDispatcher
+    TaskGraphWalker graphWalker = new TaskGraphWalker(new 
SparkDynamicPartitionPruningDispatcher());
+
+    ArrayList<Node> rootTasks = new ArrayList<>();
+    rootTasks.addAll(pctx.getRootTasks());
+    graphWalker.startWalking(rootTasks, null);
+    return pctx;
+  }
+
+  private class SparkDynamicPartitionPruningDispatcher implements Dispatcher {
+
+    @Override
+    public Object dispatch(Node nd, Stack<Node> stack, Object... nodeOutputs) 
throws SemanticException {
+      Task<? extends Serializable> task = (Task<? extends Serializable>) nd;
+
+      // If the given Task is a SparkTask then search its Work DAG for 
SparkPartitionPruningSinkOperator
+      if (task instanceof SparkTask) {
+
+        // Search for any SparkPartitionPruningSinkOperator in the SparkTask
+        for (BaseWork baseWork : ((SparkTask) task).getWork().getAllWork()) {
+          Set<Operator<?>> pruningSinkOps = OperatorUtils.getOp(baseWork, 
SparkPartitionPruningSinkOperator.class);
+
+          // For each SparkPartitionPruningSinkOperator, take the target 
MapWork and see if it is in a dependent SparkTask
+          for (Operator<?> op : pruningSinkOps) {
+            SparkPartitionPruningSinkOperator pruningSinkOp = 
(SparkPartitionPruningSinkOperator) op;
+            MapWork targetMapWork = pruningSinkOp.getConf().getTargetMapWork();
+
+            // Check if the given SparkTask has a child SparkTask that 
contains the target MapWork
+            // If it does not, then remove the DPP op
+            if (!taskContainsDependentMapWork(task, targetMapWork)) {
+              LOG.info("Disabling DPP for source work " + baseWork.getName() + 
" for target work "
+                      + targetMapWork.getName() + " as no dependency exists 
between the source and target work");
+              removeSparkPartitionPruningSink(baseWork, targetMapWork, 
pruningSinkOp);
+            }
+          }
+        }
+      }
+      return null;
+    }
+  }
+
+  /**
+   * Remove a {@link SparkPartitionPruningSinkOperator} from a given {@link 
BaseWork}. Unlink the target {@link MapWork}
+   * and the given {@link SparkPartitionPruningSinkOperator}.
+   */
+  private void removeSparkPartitionPruningSink(BaseWork sourceWork, MapWork 
targetMapWork,
+                                               
SparkPartitionPruningSinkOperator pruningSinkOp) {
+    // Remove the DPP operator subtree
+    OperatorUtils.removeBranch(pruningSinkOp);
+
+    // Remove all event source info from the target MapWork
+    String sourceWorkId = SparkUtilities.getWorkId(sourceWork);
+    SparkPartitionPruningSinkDesc pruningSinkDesc = pruningSinkOp.getConf();
+    
targetMapWork.getEventSourceTableDescMap().get(sourceWorkId).remove(pruningSinkDesc.getTable());
+    
targetMapWork.getEventSourceColumnNameMap().get(sourceWorkId).remove(pruningSinkDesc.getTargetColumnName());
+    
targetMapWork.getEventSourceColumnTypeMap().get(sourceWorkId).remove(pruningSinkDesc.getTargetColumnType());
+    
targetMapWork.getEventSourcePartKeyExprMap().get(sourceWorkId).remove(pruningSinkDesc.getPartKey());
+  }
+
+  /**
+   * Recursively go through the children of the given {@link Task} and check 
if any child {@link SparkTask} contains
+   * the specified {@link MapWork} object.
+   */
+  private boolean taskContainsDependentMapWork(Task<? extends Serializable> 
task,
+                                               MapWork work) throws 
SemanticException {
+    if (task == null || task.getChildTasks() == null) {
+      return false;
+    }
+    for (Task<? extends Serializable> childTask : task.getChildTasks()) {
+      if (childTask != null && childTask instanceof SparkTask && 
childTask.getMapWork().contains(work)) {
+        return true;
+      } else if (taskContainsDependentMapWork(childTask, work)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
index a3ec990..628726b 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.exec.ConditionalTask;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.exec.SparkHashTableSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
@@ -77,7 +78,7 @@ public class SparkMapJoinResolver implements 
PhysicalPlanResolver {
   // Check whether the specified BaseWork's operator tree contains a operator
   // of the specified operator class
   private boolean containsOp(BaseWork work, Class<?> clazz) {
-    Set<Operator<?>> matchingOps = getOp(work, clazz);
+    Set<Operator<?>> matchingOps = OperatorUtils.getOp(work, clazz);
     return matchingOps != null && !matchingOps.isEmpty();
   }
 
@@ -90,34 +91,6 @@ public class SparkMapJoinResolver implements 
PhysicalPlanResolver {
     return false;
   }
 
-  public static Set<Operator<?>> getOp(BaseWork work, Class<?> clazz) {
-    Set<Operator<?>> ops = new HashSet<Operator<?>>();
-    if (work instanceof MapWork) {
-      Collection<Operator<?>> opSet = ((MapWork) 
work).getAliasToWork().values();
-      Stack<Operator<?>> opStack = new Stack<Operator<?>>();
-      opStack.addAll(opSet);
-
-      while (!opStack.empty()) {
-        Operator<?> op = opStack.pop();
-        ops.add(op);
-        if (op.getChildOperators() != null) {
-          opStack.addAll(op.getChildOperators());
-        }
-      }
-    } else {
-      ops.addAll(work.getAllOperators());
-    }
-
-    Set<Operator<? extends OperatorDesc>> matchingOps =
-      new HashSet<Operator<? extends OperatorDesc>>();
-    for (Operator<? extends OperatorDesc> op : ops) {
-      if (clazz.isInstance(op)) {
-        matchingOps.add(op);
-      }
-    }
-    return matchingOps;
-  }
-
   @SuppressWarnings("unchecked")
   class SparkMapJoinTaskDispatcher implements Dispatcher {
 
@@ -193,7 +166,7 @@ public class SparkMapJoinResolver implements 
PhysicalPlanResolver {
               containsOp(work, MapJoinOperator.class)) {
             work.setMapRedLocalWork(new MapredLocalWork());
           }
-          Set<Operator<?>> ops = getOp(work, MapJoinOperator.class);
+          Set<Operator<?>> ops = OperatorUtils.getOp(work, 
MapJoinOperator.class);
           if (ops == null || ops.isEmpty()) {
             continue;
           }
@@ -223,7 +196,7 @@ public class SparkMapJoinResolver implements 
PhysicalPlanResolver {
 
           for (BaseWork parentWork : originalWork.getParents(work)) {
             Set<Operator<?>> hashTableSinkOps =
-                getOp(parentWork, SparkHashTableSinkOperator.class);
+                OperatorUtils.getOp(parentWork, 
SparkHashTableSinkOperator.class);
             if (hashTableSinkOps == null || hashTableSinkOps.isEmpty()) {
               continue;
             }

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java
index 0ca8c02..baf85cf 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.plan.AbstractOperatorDesc;
 import org.apache.hadoop.hive.ql.plan.Explain;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 
 @Explain(displayName = "Spark Partition Pruning Sink Operator")
@@ -43,7 +44,7 @@ public class SparkPartitionPruningSinkDesc extends 
AbstractOperatorDesc {
 
   private Path path;
 
-  private String targetWork;
+  private MapWork targetMapWork;
 
   @Explain(displayName = "tmp Path", explainLevels = { Explain.Level.EXTENDED 
})
   public Path getPath() {
@@ -56,11 +57,15 @@ public class SparkPartitionPruningSinkDesc extends 
AbstractOperatorDesc {
 
   @Explain(displayName = "target work")
   public String getTargetWork() {
-    return this.targetWork;
+    return this.targetMapWork.getName();
   }
 
-  public void setTargetWork(String targetWork) {
-    this.targetWork = targetWork;
+  public MapWork getTargetMapWork() {
+    return this.targetMapWork;
+  }
+
+  public void setTargetMapWork(MapWork targetMapWork) {
+    this.targetMapWork = targetMapWork;
   }
 
   public TableScanOperator getTableScan() {

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
index abc9fcf..e763302 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.exec.CommonJoinOperator;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
@@ -38,7 +39,6 @@ import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
 import org.apache.hadoop.hive.ql.optimizer.physical.GenMRSkewJoinProcessor;
 import org.apache.hadoop.hive.ql.optimizer.physical.GenSparkSkewJoinProcessor;
 import org.apache.hadoop.hive.ql.optimizer.physical.SkewJoinProcFactory;
-import org.apache.hadoop.hive.ql.optimizer.physical.SparkMapJoinResolver;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.spark.GenSparkUtils;
@@ -105,7 +105,7 @@ public class SparkSkewJoinProcFactory {
       ParseContext parseContext) throws SemanticException {
     SparkWork currentWork = currentTask.getWork();
     Set<Operator<?>> reduceSinkSet =
-        SparkMapJoinResolver.getOp(reduceWork, ReduceSinkOperator.class);
+        OperatorUtils.getOp(reduceWork, ReduceSinkOperator.class);
     if (currentWork.getChildren(reduceWork).size() == 1 && 
canSplit(currentWork)
       && reduceSinkSet.size() == 1) {
       ReduceSinkOperator reduceSink = (ReduceSinkOperator) 
reduceSinkSet.iterator().next();
@@ -231,7 +231,7 @@ public class SparkSkewJoinProcFactory {
       List<Task<? extends Serializable>> children = currTask.getChildTasks();
       return !joinOp.getConf().isFixedAsSorted() && 
sparkWork.contains(reduceWork) &&
           (children == null || children.size() <= 1) &&
-          SparkMapJoinResolver.getOp(reduceWork, 
CommonJoinOperator.class).size() == 1;
+          OperatorUtils.getOp(reduceWork, CommonJoinOperator.class).size() == 
1;
     }
     return false;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
index 4f8ba6f..4078d2a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
@@ -478,7 +478,7 @@ public class GenSparkUtils {
     }
 
     desc.setPath(new Path(tmpPath, sourceId));
-    desc.setTargetWork(targetWork.getName());
+    desc.setTargetMapWork(targetWork);
 
     // store table descriptor in map-targetWork
     if (!targetWork.getEventSourceTableDescMap().containsKey(sourceId)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
index ab8db20..8144350 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
@@ -71,6 +71,7 @@ import 
org.apache.hadoop.hive.ql.optimizer.physical.MetadataOnlyOptimizer;
 import org.apache.hadoop.hive.ql.optimizer.physical.NullScanOptimizer;
 import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext;
 import org.apache.hadoop.hive.ql.optimizer.physical.SparkCrossProductCheck;
+import 
org.apache.hadoop.hive.ql.optimizer.physical.SparkDynamicPartitionPruningResolver;
 import org.apache.hadoop.hive.ql.optimizer.physical.SparkMapJoinResolver;
 import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger;
 import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
@@ -563,6 +564,10 @@ public class SparkCompiler extends TaskCompiler {
 
     physicalCtx = new SparkMapJoinResolver().resolve(physicalCtx);
 
+    if (conf.isSparkDPPAny()) {
+      physicalCtx = new 
SparkDynamicPartitionPruningResolver().resolve(physicalCtx);
+    }
+
     if (conf.getBoolVar(HiveConf.ConfVars.HIVENULLSCANOPTIMIZE)) {
       physicalCtx = new NullScanOptimizer().resolve(physicalCtx);
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_recursive_mapjoin.q
----------------------------------------------------------------------
diff --git 
a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_recursive_mapjoin.q
 
b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_recursive_mapjoin.q
new file mode 100644
index 0000000..5e19b97
--- /dev/null
+++ 
b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_recursive_mapjoin.q
@@ -0,0 +1,147 @@
+SET hive.spark.dynamic.partition.pruning=true;
+SET hive.auto.convert.join=true;
+SET hive.strict.checks.cartesian.product=false;
+
+CREATE TABLE part_table1 (col int) PARTITIONED BY (part1_col int);
+CREATE TABLE part_table2 (col int) PARTITIONED BY (part2_col int);
+CREATE TABLE part_table3 (col int) PARTITIONED BY (part3_col int);
+CREATE TABLE part_table4 (col int) PARTITIONED BY (part4_col int);
+CREATE TABLE part_table5 (col int) PARTITIONED BY (part5_col int);
+
+CREATE TABLE reg_table (col int);
+
+ALTER TABLE part_table1 ADD PARTITION (part1_col = 1);
+
+ALTER TABLE part_table2 ADD PARTITION (part2_col = 1);
+ALTER TABLE part_table2 ADD PARTITION (part2_col = 2);
+
+ALTER TABLE part_table3 ADD PARTITION (part3_col = 1);
+ALTER TABLE part_table3 ADD PARTITION (part3_col = 2);
+ALTER TABLE part_table3 ADD PARTITION (part3_col = 3);
+
+ALTER TABLE part_table4 ADD PARTITION (part4_col = 1);
+ALTER TABLE part_table4 ADD PARTITION (part4_col = 2);
+ALTER TABLE part_table4 ADD PARTITION (part4_col = 3);
+ALTER TABLE part_table4 ADD PARTITION (part4_col = 4);
+
+ALTER TABLE part_table5 ADD PARTITION (part5_col = 1);
+ALTER TABLE part_table5 ADD PARTITION (part5_col = 2);
+ALTER TABLE part_table5 ADD PARTITION (part5_col = 3);
+ALTER TABLE part_table5 ADD PARTITION (part5_col = 4);
+ALTER TABLE part_table5 ADD PARTITION (part5_col = 5);
+
+INSERT INTO TABLE part_table1 PARTITION (part1_col = 1) VALUES (1);
+
+INSERT INTO TABLE part_table2 PARTITION (part2_col = 1) VALUES (1);
+INSERT INTO TABLE part_table2 PARTITION (part2_col = 2) VALUES (2);
+
+INSERT INTO TABLE part_table3 PARTITION (part3_col = 1) VALUES (1);
+INSERT INTO TABLE part_table3 PARTITION (part3_col = 2) VALUES (2);
+INSERT INTO TABLE part_table3 PARTITION (part3_col = 3) VALUES (3);
+
+INSERT INTO TABLE part_table4 PARTITION (part4_col = 1) VALUES (1);
+INSERT INTO TABLE part_table4 PARTITION (part4_col = 2) VALUES (2);
+INSERT INTO TABLE part_table4 PARTITION (part4_col = 3) VALUES (3);
+INSERT INTO TABLE part_table4 PARTITION (part4_col = 4) VALUES (4);
+
+INSERT INTO TABLE part_table5 PARTITION (part5_col = 1) VALUES (1);
+INSERT INTO TABLE part_table5 PARTITION (part5_col = 2) VALUES (2);
+INSERT INTO TABLE part_table5 PARTITION (part5_col = 3) VALUES (3);
+INSERT INTO TABLE part_table5 PARTITION (part5_col = 4) VALUES (4);
+INSERT INTO TABLE part_table5 PARTITION (part5_col = 5) VALUES (5);
+
+INSERT INTO table reg_table VALUES (1), (2), (3), (4), (5), (6);
+
+-- 3 table join pt2 pruned based on scan from pt1
+explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col;
+
+SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col;
+
+-- 4 table join pt3 pruned based on pt2, pt2 pruned based on pt1
+explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col;
+
+SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col;
+
+-- 5 table join pt4 pruned based on pt3, pt3 pruned based on pt2, pt2 pruned 
based on pt1
+explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               part_table4 pt4,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col
+        AND    pt4.part4_col = pt1.part1_col;
+
+SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       part_table4 pt4,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col
+AND    pt4.part4_col = pt1.part1_col;
+
+-- 6 table join pt5 pruned based on pt4, pt4 pruned based on pt3, pt3 pruned 
based on pt2,
+-- pt2 pruned based on pt1
+explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               part_table4 pt4,
+               part_table5 pt5,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col
+        AND    pt4.part4_col = pt1.part1_col
+        AND    pt5.part5_col = pt1.part1_col;
+
+SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       part_table4 pt4,
+       part_table5 pt5,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col
+AND    pt4.part4_col = pt1.part1_col
+AND    pt5.part5_col = pt1.part1_col;
+
+-- Cleanup
+DROP TABLE part_table1;
+DROP TABLE part_table2;
+DROP TABLE part_table3;
+DROP TABLE part_table4;
+DROP TABLE part_table5;
+
+DROP TABLE reg_table;

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q
----------------------------------------------------------------------
diff --git 
a/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q
 
b/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q
index b4ecbef..9d934a4 100644
--- 
a/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q
+++ 
b/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q
@@ -155,8 +155,7 @@ select count(*) from srcpart where ds = '2008-04-08' and hr 
= 11;
 
 -- empty set
 EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = 
srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST';
--- Disabled until TEZ-1486 is fixed
--- select count(*) from srcpart join srcpart_date on (srcpart.ds = 
srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST';
+select count(*) from srcpart join srcpart_date on (srcpart.ds = 
srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST';
 
 -- expressions
 EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = 
cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11;
@@ -184,9 +183,8 @@ select count(*) from srcpart join srcpart_date on 
(srcpart.ds = srcpart_date.ds)
 where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and 
srcpart.hr = 11;
 EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = 
srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) 
 where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13;
--- Disabled until TEZ-1486 is fixed
--- select count(*) from srcpart join srcpart_date on (srcpart.ds = 
srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) 
--- where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13;
+select count(*) from srcpart join srcpart_date on (srcpart.ds = 
srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) 
+where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13;
 
 -- union + subquery
 EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select 
max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart);

http://git-wip-us.apache.org/repos/asf/hive/blob/90e68288/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_recursive_mapjoin.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_recursive_mapjoin.q.out
 
b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_recursive_mapjoin.q.out
new file mode 100644
index 0000000..4e37daf
--- /dev/null
+++ 
b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_recursive_mapjoin.q.out
@@ -0,0 +1,1097 @@
+PREHOOK: query: CREATE TABLE part_table1 (col int) PARTITIONED BY (part1_col 
int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_table1
+POSTHOOK: query: CREATE TABLE part_table1 (col int) PARTITIONED BY (part1_col 
int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_table1
+PREHOOK: query: CREATE TABLE part_table2 (col int) PARTITIONED BY (part2_col 
int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_table2
+POSTHOOK: query: CREATE TABLE part_table2 (col int) PARTITIONED BY (part2_col 
int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_table2
+PREHOOK: query: CREATE TABLE part_table3 (col int) PARTITIONED BY (part3_col 
int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_table3
+POSTHOOK: query: CREATE TABLE part_table3 (col int) PARTITIONED BY (part3_col 
int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_table3
+PREHOOK: query: CREATE TABLE part_table4 (col int) PARTITIONED BY (part4_col 
int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_table4
+POSTHOOK: query: CREATE TABLE part_table4 (col int) PARTITIONED BY (part4_col 
int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_table4
+PREHOOK: query: CREATE TABLE part_table5 (col int) PARTITIONED BY (part5_col 
int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_table5
+POSTHOOK: query: CREATE TABLE part_table5 (col int) PARTITIONED BY (part5_col 
int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_table5
+PREHOOK: query: CREATE TABLE reg_table (col int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@reg_table
+POSTHOOK: query: CREATE TABLE reg_table (col int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@reg_table
+PREHOOK: query: ALTER TABLE part_table1 ADD PARTITION (part1_col = 1)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table1
+POSTHOOK: query: ALTER TABLE part_table1 ADD PARTITION (part1_col = 1)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table1
+POSTHOOK: Output: default@part_table1@part1_col=1
+PREHOOK: query: ALTER TABLE part_table2 ADD PARTITION (part2_col = 1)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table2
+POSTHOOK: query: ALTER TABLE part_table2 ADD PARTITION (part2_col = 1)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table2
+POSTHOOK: Output: default@part_table2@part2_col=1
+PREHOOK: query: ALTER TABLE part_table2 ADD PARTITION (part2_col = 2)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table2
+POSTHOOK: query: ALTER TABLE part_table2 ADD PARTITION (part2_col = 2)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table2
+POSTHOOK: Output: default@part_table2@part2_col=2
+PREHOOK: query: ALTER TABLE part_table3 ADD PARTITION (part3_col = 1)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table3
+POSTHOOK: query: ALTER TABLE part_table3 ADD PARTITION (part3_col = 1)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table3
+POSTHOOK: Output: default@part_table3@part3_col=1
+PREHOOK: query: ALTER TABLE part_table3 ADD PARTITION (part3_col = 2)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table3
+POSTHOOK: query: ALTER TABLE part_table3 ADD PARTITION (part3_col = 2)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table3
+POSTHOOK: Output: default@part_table3@part3_col=2
+PREHOOK: query: ALTER TABLE part_table3 ADD PARTITION (part3_col = 3)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table3
+POSTHOOK: query: ALTER TABLE part_table3 ADD PARTITION (part3_col = 3)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table3
+POSTHOOK: Output: default@part_table3@part3_col=3
+PREHOOK: query: ALTER TABLE part_table4 ADD PARTITION (part4_col = 1)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table4
+POSTHOOK: query: ALTER TABLE part_table4 ADD PARTITION (part4_col = 1)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table4
+POSTHOOK: Output: default@part_table4@part4_col=1
+PREHOOK: query: ALTER TABLE part_table4 ADD PARTITION (part4_col = 2)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table4
+POSTHOOK: query: ALTER TABLE part_table4 ADD PARTITION (part4_col = 2)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table4
+POSTHOOK: Output: default@part_table4@part4_col=2
+PREHOOK: query: ALTER TABLE part_table4 ADD PARTITION (part4_col = 3)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table4
+POSTHOOK: query: ALTER TABLE part_table4 ADD PARTITION (part4_col = 3)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table4
+POSTHOOK: Output: default@part_table4@part4_col=3
+PREHOOK: query: ALTER TABLE part_table4 ADD PARTITION (part4_col = 4)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table4
+POSTHOOK: query: ALTER TABLE part_table4 ADD PARTITION (part4_col = 4)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table4
+POSTHOOK: Output: default@part_table4@part4_col=4
+PREHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 1)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table5
+POSTHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 1)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table5
+POSTHOOK: Output: default@part_table5@part5_col=1
+PREHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 2)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table5
+POSTHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 2)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table5
+POSTHOOK: Output: default@part_table5@part5_col=2
+PREHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 3)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table5
+POSTHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 3)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table5
+POSTHOOK: Output: default@part_table5@part5_col=3
+PREHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 4)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table5
+POSTHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 4)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table5
+POSTHOOK: Output: default@part_table5@part5_col=4
+PREHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 5)
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@part_table5
+POSTHOOK: query: ALTER TABLE part_table5 ADD PARTITION (part5_col = 5)
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@part_table5
+POSTHOOK: Output: default@part_table5@part5_col=5
+PREHOOK: query: INSERT INTO TABLE part_table1 PARTITION (part1_col = 1) VALUES 
(1)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table1@part1_col=1
+POSTHOOK: query: INSERT INTO TABLE part_table1 PARTITION (part1_col = 1) 
VALUES (1)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table1@part1_col=1
+POSTHOOK: Lineage: part_table1 PARTITION(part1_col=1).col EXPRESSION 
[(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table2 PARTITION (part2_col = 1) VALUES 
(1)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table2@part2_col=1
+POSTHOOK: query: INSERT INTO TABLE part_table2 PARTITION (part2_col = 1) 
VALUES (1)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table2@part2_col=1
+POSTHOOK: Lineage: part_table2 PARTITION(part2_col=1).col EXPRESSION 
[(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table2 PARTITION (part2_col = 2) VALUES 
(2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table2@part2_col=2
+POSTHOOK: query: INSERT INTO TABLE part_table2 PARTITION (part2_col = 2) 
VALUES (2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table2@part2_col=2
+POSTHOOK: Lineage: part_table2 PARTITION(part2_col=2).col EXPRESSION 
[(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table3 PARTITION (part3_col = 1) VALUES 
(1)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table3@part3_col=1
+POSTHOOK: query: INSERT INTO TABLE part_table3 PARTITION (part3_col = 1) 
VALUES (1)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table3@part3_col=1
+POSTHOOK: Lineage: part_table3 PARTITION(part3_col=1).col EXPRESSION 
[(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table3 PARTITION (part3_col = 2) VALUES 
(2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table3@part3_col=2
+POSTHOOK: query: INSERT INTO TABLE part_table3 PARTITION (part3_col = 2) 
VALUES (2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table3@part3_col=2
+POSTHOOK: Lineage: part_table3 PARTITION(part3_col=2).col EXPRESSION 
[(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table3 PARTITION (part3_col = 3) VALUES 
(3)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table3@part3_col=3
+POSTHOOK: query: INSERT INTO TABLE part_table3 PARTITION (part3_col = 3) 
VALUES (3)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table3@part3_col=3
+POSTHOOK: Lineage: part_table3 PARTITION(part3_col=3).col EXPRESSION 
[(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table4 PARTITION (part4_col = 1) VALUES 
(1)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table4@part4_col=1
+POSTHOOK: query: INSERT INTO TABLE part_table4 PARTITION (part4_col = 1) 
VALUES (1)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table4@part4_col=1
+POSTHOOK: Lineage: part_table4 PARTITION(part4_col=1).col EXPRESSION 
[(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table4 PARTITION (part4_col = 2) VALUES 
(2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table4@part4_col=2
+POSTHOOK: query: INSERT INTO TABLE part_table4 PARTITION (part4_col = 2) 
VALUES (2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table4@part4_col=2
+POSTHOOK: Lineage: part_table4 PARTITION(part4_col=2).col EXPRESSION 
[(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table4 PARTITION (part4_col = 3) VALUES 
(3)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table4@part4_col=3
+POSTHOOK: query: INSERT INTO TABLE part_table4 PARTITION (part4_col = 3) 
VALUES (3)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table4@part4_col=3
+POSTHOOK: Lineage: part_table4 PARTITION(part4_col=3).col EXPRESSION 
[(values__tmp__table__9)values__tmp__table__9.FieldSchema(name:tmp_values_col1, 
type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table4 PARTITION (part4_col = 4) VALUES 
(4)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table4@part4_col=4
+POSTHOOK: query: INSERT INTO TABLE part_table4 PARTITION (part4_col = 4) 
VALUES (4)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table4@part4_col=4
+POSTHOOK: Lineage: part_table4 PARTITION(part4_col=4).col EXPRESSION 
[(values__tmp__table__10)values__tmp__table__10.FieldSchema(name:tmp_values_col1,
 type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 1) VALUES 
(1)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table5@part5_col=1
+POSTHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 1) 
VALUES (1)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table5@part5_col=1
+POSTHOOK: Lineage: part_table5 PARTITION(part5_col=1).col EXPRESSION 
[(values__tmp__table__11)values__tmp__table__11.FieldSchema(name:tmp_values_col1,
 type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 2) VALUES 
(2)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table5@part5_col=2
+POSTHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 2) 
VALUES (2)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table5@part5_col=2
+POSTHOOK: Lineage: part_table5 PARTITION(part5_col=2).col EXPRESSION 
[(values__tmp__table__12)values__tmp__table__12.FieldSchema(name:tmp_values_col1,
 type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 3) VALUES 
(3)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table5@part5_col=3
+POSTHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 3) 
VALUES (3)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table5@part5_col=3
+POSTHOOK: Lineage: part_table5 PARTITION(part5_col=3).col EXPRESSION 
[(values__tmp__table__13)values__tmp__table__13.FieldSchema(name:tmp_values_col1,
 type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 4) VALUES 
(4)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table5@part5_col=4
+POSTHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 4) 
VALUES (4)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table5@part5_col=4
+POSTHOOK: Lineage: part_table5 PARTITION(part5_col=4).col EXPRESSION 
[(values__tmp__table__14)values__tmp__table__14.FieldSchema(name:tmp_values_col1,
 type:string, comment:), ]
+PREHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 5) VALUES 
(5)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@part_table5@part5_col=5
+POSTHOOK: query: INSERT INTO TABLE part_table5 PARTITION (part5_col = 5) 
VALUES (5)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@part_table5@part5_col=5
+POSTHOOK: Lineage: part_table5 PARTITION(part5_col=5).col EXPRESSION 
[(values__tmp__table__15)values__tmp__table__15.FieldSchema(name:tmp_values_col1,
 type:string, comment:), ]
+PREHOOK: query: INSERT INTO table reg_table VALUES (1), (2), (3), (4), (5), (6)
+PREHOOK: type: QUERY
+PREHOOK: Output: default@reg_table
+POSTHOOK: query: INSERT INTO table reg_table VALUES (1), (2), (3), (4), (5), 
(6)
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@reg_table
+POSTHOOK: Lineage: reg_table.col EXPRESSION 
[(values__tmp__table__16)values__tmp__table__16.FieldSchema(name:tmp_values_col1,
 type:string, comment:), ]
+PREHOOK: query: explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+PREHOOK: type: QUERY
+POSTHOOK: query: explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: pt2
+                  Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part2_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: pt1
+                  Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part1_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: rt
+                  Statistics: Num rows: 6 Data size: 6 Basic stats: COMPLETE 
Column stats: NONE
+                  Filter Operator
+                    predicate: col is not null (type: boolean)
+                    Statistics: Num rows: 6 Data size: 6 Basic stats: COMPLETE 
Column stats: NONE
+                    Select Operator
+                      expressions: col (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 6 Data size: 6 Basic stats: 
COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                             Inner Join 1 to 2
+                        keys:
+                          0 _col1 (type: int)
+                          1 _col1 (type: int)
+                          2 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                        input vertices:
+                          0 Map 1
+                          1 Map 2
+                        Statistics: Num rows: 13 Data size: 13 Basic stats: 
COMPLETE Column stats: NONE
+                        Select Operator
+                          expressions: _col2 (type: int), _col3 (type: int), 
_col0 (type: int), _col1 (type: int), _col4 (type: int)
+                          outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                          Statistics: Num rows: 13 Data size: 13 Basic stats: 
COMPLETE Column stats: NONE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 13 Data size: 13 Basic 
stats: COMPLETE Column stats: NONE
+                            table:
+                                input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: 
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_table1
+PREHOOK: Input: default@part_table1@part1_col=1
+PREHOOK: Input: default@part_table2
+PREHOOK: Input: default@part_table2@part2_col=1
+PREHOOK: Input: default@part_table2@part2_col=2
+PREHOOK: Input: default@reg_table
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_table1
+POSTHOOK: Input: default@part_table1@part1_col=1
+POSTHOOK: Input: default@part_table2
+POSTHOOK: Input: default@part_table2@part2_col=1
+POSTHOOK: Input: default@part_table2@part2_col=2
+POSTHOOK: Input: default@reg_table
+#### A masked pattern was here ####
+1      1       1       1       1
+PREHOOK: query: explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col
+PREHOOK: type: QUERY
+POSTHOOK: query: explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: pt2
+                  Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part2_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: pt1
+                  Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part1_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: pt3
+                  Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part3_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: rt
+                  Statistics: Num rows: 6 Data size: 6 Basic stats: COMPLETE 
Column stats: NONE
+                  Filter Operator
+                    predicate: col is not null (type: boolean)
+                    Statistics: Num rows: 6 Data size: 6 Basic stats: COMPLETE 
Column stats: NONE
+                    Select Operator
+                      expressions: col (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 6 Data size: 6 Basic stats: 
COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                             Inner Join 1 to 2
+                             Inner Join 1 to 3
+                        keys:
+                          0 _col1 (type: int)
+                          1 _col1 (type: int)
+                          2 _col1 (type: int)
+                          3 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6
+                        input vertices:
+                          0 Map 1
+                          1 Map 2
+                          2 Map 3
+                        Statistics: Num rows: 19 Data size: 19 Basic stats: 
COMPLETE Column stats: NONE
+                        Select Operator
+                          expressions: _col2 (type: int), _col3 (type: int), 
_col0 (type: int), _col1 (type: int), _col4 (type: int), _col5 (type: int), 
_col6 (type: int)
+                          outputColumnNames: _col0, _col1, _col2, _col3, 
_col4, _col5, _col6
+                          Statistics: Num rows: 19 Data size: 19 Basic stats: 
COMPLETE Column stats: NONE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 19 Data size: 19 Basic 
stats: COMPLETE Column stats: NONE
+                            table:
+                                input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: 
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_table1
+PREHOOK: Input: default@part_table1@part1_col=1
+PREHOOK: Input: default@part_table2
+PREHOOK: Input: default@part_table2@part2_col=1
+PREHOOK: Input: default@part_table2@part2_col=2
+PREHOOK: Input: default@part_table3
+PREHOOK: Input: default@part_table3@part3_col=1
+PREHOOK: Input: default@part_table3@part3_col=2
+PREHOOK: Input: default@part_table3@part3_col=3
+PREHOOK: Input: default@reg_table
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_table1
+POSTHOOK: Input: default@part_table1@part1_col=1
+POSTHOOK: Input: default@part_table2
+POSTHOOK: Input: default@part_table2@part2_col=1
+POSTHOOK: Input: default@part_table2@part2_col=2
+POSTHOOK: Input: default@part_table3
+POSTHOOK: Input: default@part_table3@part3_col=1
+POSTHOOK: Input: default@part_table3@part3_col=2
+POSTHOOK: Input: default@part_table3@part3_col=3
+POSTHOOK: Input: default@reg_table
+#### A masked pattern was here ####
+1      1       1       1       1       1       1
+PREHOOK: query: explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               part_table4 pt4,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col
+        AND    pt4.part4_col = pt1.part1_col
+PREHOOK: type: QUERY
+POSTHOOK: query: explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               part_table4 pt4,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col
+        AND    pt4.part4_col = pt1.part1_col
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: pt2
+                  Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part2_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: pt1
+                  Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part1_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: pt3
+                  Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part3_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: pt4
+                  Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part4_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: rt
+                  Statistics: Num rows: 6 Data size: 6 Basic stats: COMPLETE 
Column stats: NONE
+                  Filter Operator
+                    predicate: col is not null (type: boolean)
+                    Statistics: Num rows: 6 Data size: 6 Basic stats: COMPLETE 
Column stats: NONE
+                    Select Operator
+                      expressions: col (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 6 Data size: 6 Basic stats: 
COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                             Inner Join 1 to 2
+                             Inner Join 1 to 3
+                             Inner Join 1 to 4
+                        keys:
+                          0 _col1 (type: int)
+                          1 _col1 (type: int)
+                          2 _col1 (type: int)
+                          3 _col1 (type: int)
+                          4 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6, _col7, _col8
+                        input vertices:
+                          0 Map 1
+                          1 Map 2
+                          2 Map 3
+                          3 Map 4
+                        Statistics: Num rows: 26 Data size: 26 Basic stats: 
COMPLETE Column stats: NONE
+                        Select Operator
+                          expressions: _col2 (type: int), _col3 (type: int), 
_col0 (type: int), _col1 (type: int), _col4 (type: int), _col5 (type: int), 
_col6 (type: int), _col7 (type: int), _col8 (type: int)
+                          outputColumnNames: _col0, _col1, _col2, _col3, 
_col4, _col5, _col6, _col7, _col8
+                          Statistics: Num rows: 26 Data size: 26 Basic stats: 
COMPLETE Column stats: NONE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 26 Data size: 26 Basic 
stats: COMPLETE Column stats: NONE
+                            table:
+                                input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: 
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       part_table4 pt4,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col
+AND    pt4.part4_col = pt1.part1_col
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_table1
+PREHOOK: Input: default@part_table1@part1_col=1
+PREHOOK: Input: default@part_table2
+PREHOOK: Input: default@part_table2@part2_col=1
+PREHOOK: Input: default@part_table2@part2_col=2
+PREHOOK: Input: default@part_table3
+PREHOOK: Input: default@part_table3@part3_col=1
+PREHOOK: Input: default@part_table3@part3_col=2
+PREHOOK: Input: default@part_table3@part3_col=3
+PREHOOK: Input: default@part_table4
+PREHOOK: Input: default@part_table4@part4_col=1
+PREHOOK: Input: default@part_table4@part4_col=2
+PREHOOK: Input: default@part_table4@part4_col=3
+PREHOOK: Input: default@part_table4@part4_col=4
+PREHOOK: Input: default@reg_table
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       part_table4 pt4,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col
+AND    pt4.part4_col = pt1.part1_col
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_table1
+POSTHOOK: Input: default@part_table1@part1_col=1
+POSTHOOK: Input: default@part_table2
+POSTHOOK: Input: default@part_table2@part2_col=1
+POSTHOOK: Input: default@part_table2@part2_col=2
+POSTHOOK: Input: default@part_table3
+POSTHOOK: Input: default@part_table3@part3_col=1
+POSTHOOK: Input: default@part_table3@part3_col=2
+POSTHOOK: Input: default@part_table3@part3_col=3
+POSTHOOK: Input: default@part_table4
+POSTHOOK: Input: default@part_table4@part4_col=1
+POSTHOOK: Input: default@part_table4@part4_col=2
+POSTHOOK: Input: default@part_table4@part4_col=3
+POSTHOOK: Input: default@part_table4@part4_col=4
+POSTHOOK: Input: default@reg_table
+#### A masked pattern was here ####
+1      1       1       1       1       1       1       1       1
+PREHOOK: query: explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               part_table4 pt4,
+               part_table5 pt5,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col
+        AND    pt4.part4_col = pt1.part1_col
+        AND    pt5.part5_col = pt1.part1_col
+PREHOOK: type: QUERY
+POSTHOOK: query: explain SELECT *
+        FROM   part_table1 pt1,
+               part_table2 pt2,
+               part_table3 pt3,
+               part_table4 pt4,
+               part_table5 pt5,
+               reg_table rt
+        WHERE  rt.col = pt1.part1_col
+        AND    pt2.part2_col = pt1.part1_col
+        AND    pt3.part3_col = pt1.part1_col
+        AND    pt4.part4_col = pt1.part1_col
+        AND    pt5.part5_col = pt1.part1_col
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-1 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: pt2
+                  Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part2_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col1 (type: int)
+                        5 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: pt1
+                  Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part1_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col1 (type: int)
+                        5 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 3 
+            Map Operator Tree:
+                TableScan
+                  alias: pt3
+                  Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part3_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col1 (type: int)
+                        5 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: pt4
+                  Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part4_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col1 (type: int)
+                        5 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: pt5
+                  Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE 
Column stats: NONE
+                  Select Operator
+                    expressions: col (type: int), part5_col (type: int)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE 
Column stats: NONE
+                    Spark HashTable Sink Operator
+                      keys:
+                        0 _col1 (type: int)
+                        1 _col1 (type: int)
+                        2 _col1 (type: int)
+                        3 _col1 (type: int)
+                        4 _col1 (type: int)
+                        5 _col0 (type: int)
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: rt
+                  Statistics: Num rows: 6 Data size: 6 Basic stats: COMPLETE 
Column stats: NONE
+                  Filter Operator
+                    predicate: col is not null (type: boolean)
+                    Statistics: Num rows: 6 Data size: 6 Basic stats: COMPLETE 
Column stats: NONE
+                    Select Operator
+                      expressions: col (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 6 Data size: 6 Basic stats: 
COMPLETE Column stats: NONE
+                      Map Join Operator
+                        condition map:
+                             Inner Join 0 to 1
+                             Inner Join 1 to 2
+                             Inner Join 1 to 3
+                             Inner Join 1 to 4
+                             Inner Join 1 to 5
+                        keys:
+                          0 _col1 (type: int)
+                          1 _col1 (type: int)
+                          2 _col1 (type: int)
+                          3 _col1 (type: int)
+                          4 _col1 (type: int)
+                          5 _col0 (type: int)
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6, _col7, _col8, _col9, _col10
+                        input vertices:
+                          0 Map 1
+                          1 Map 2
+                          2 Map 3
+                          3 Map 4
+                          4 Map 5
+                        Statistics: Num rows: 33 Data size: 33 Basic stats: 
COMPLETE Column stats: NONE
+                        Select Operator
+                          expressions: _col2 (type: int), _col3 (type: int), 
_col0 (type: int), _col1 (type: int), _col4 (type: int), _col5 (type: int), 
_col6 (type: int), _col7 (type: int), _col8 (type: int), _col9 (type: int), 
_col10 (type: int)
+                          outputColumnNames: _col0, _col1, _col2, _col3, 
_col4, _col5, _col6, _col7, _col8, _col9, _col10
+                          Statistics: Num rows: 33 Data size: 33 Basic stats: 
COMPLETE Column stats: NONE
+                          File Output Operator
+                            compressed: false
+                            Statistics: Num rows: 33 Data size: 33 Basic 
stats: COMPLETE Column stats: NONE
+                            table:
+                                input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                                output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                                serde: 
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Local Work:
+              Map Reduce Local Work
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       part_table4 pt4,
+       part_table5 pt5,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col
+AND    pt4.part4_col = pt1.part1_col
+AND    pt5.part5_col = pt1.part1_col
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_table1
+PREHOOK: Input: default@part_table1@part1_col=1
+PREHOOK: Input: default@part_table2
+PREHOOK: Input: default@part_table2@part2_col=1
+PREHOOK: Input: default@part_table2@part2_col=2
+PREHOOK: Input: default@part_table3
+PREHOOK: Input: default@part_table3@part3_col=1
+PREHOOK: Input: default@part_table3@part3_col=2
+PREHOOK: Input: default@part_table3@part3_col=3
+PREHOOK: Input: default@part_table4
+PREHOOK: Input: default@part_table4@part4_col=1
+PREHOOK: Input: default@part_table4@part4_col=2
+PREHOOK: Input: default@part_table4@part4_col=3
+PREHOOK: Input: default@part_table4@part4_col=4
+PREHOOK: Input: default@part_table5
+PREHOOK: Input: default@part_table5@part5_col=1
+PREHOOK: Input: default@part_table5@part5_col=2
+PREHOOK: Input: default@part_table5@part5_col=3
+PREHOOK: Input: default@part_table5@part5_col=4
+PREHOOK: Input: default@part_table5@part5_col=5
+PREHOOK: Input: default@reg_table
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT *
+FROM   part_table1 pt1,
+       part_table2 pt2,
+       part_table3 pt3,
+       part_table4 pt4,
+       part_table5 pt5,
+       reg_table rt
+WHERE  rt.col = pt1.part1_col
+AND    pt2.part2_col = pt1.part1_col
+AND    pt3.part3_col = pt1.part1_col
+AND    pt4.part4_col = pt1.part1_col
+AND    pt5.part5_col = pt1.part1_col
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_table1
+POSTHOOK: Input: default@part_table1@part1_col=1
+POSTHOOK: Input: default@part_table2
+POSTHOOK: Input: default@part_table2@part2_col=1
+POSTHOOK: Input: default@part_table2@part2_col=2
+POSTHOOK: Input: default@part_table3
+POSTHOOK: Input: default@part_table3@part3_col=1
+POSTHOOK: Input: default@part_table3@part3_col=2
+POSTHOOK: Input: default@part_table3@part3_col=3
+POSTHOOK: Input: default@part_table4
+POSTHOOK: Input: default@part_table4@part4_col=1
+POSTHOOK: Input: default@part_table4@part4_col=2
+POSTHOOK: Input: default@part_table4@part4_col=3
+POSTHOOK: Input: default@part_table4@part4_col=4
+POSTHOOK: Input: default@part_table5
+POSTHOOK: Input: default@part_table5@part5_col=1
+POSTHOOK: Input: default@part_table5@part5_col=2
+POSTHOOK: Input: default@part_table5@part5_col=3
+POSTHOOK: Input: default@part_table5@part5_col=4
+POSTHOOK: Input: default@part_table5@part5_col=5
+POSTHOOK: Input: default@reg_table
+#### A masked pattern was here ####
+1      1       1       1       1       1       1       1       1       1       
1
+PREHOOK: query: DROP TABLE part_table1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@part_table1
+PREHOOK: Output: default@part_table1
+POSTHOOK: query: DROP TABLE part_table1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@part_table1
+POSTHOOK: Output: default@part_table1
+PREHOOK: query: DROP TABLE part_table2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@part_table2
+PREHOOK: Output: default@part_table2
+POSTHOOK: query: DROP TABLE part_table2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@part_table2
+POSTHOOK: Output: default@part_table2
+PREHOOK: query: DROP TABLE part_table3
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@part_table3
+PREHOOK: Output: default@part_table3
+POSTHOOK: query: DROP TABLE part_table3
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@part_table3
+POSTHOOK: Output: default@part_table3
+PREHOOK: query: DROP TABLE part_table4
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@part_table4
+PREHOOK: Output: default@part_table4
+POSTHOOK: query: DROP TABLE part_table4
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@part_table4
+POSTHOOK: Output: default@part_table4
+PREHOOK: query: DROP TABLE part_table5
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@part_table5
+PREHOOK: Output: default@part_table5
+POSTHOOK: query: DROP TABLE part_table5
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@part_table5
+POSTHOOK: Output: default@part_table5
+PREHOOK: query: DROP TABLE reg_table
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@reg_table
+PREHOOK: Output: default@reg_table
+POSTHOOK: query: DROP TABLE reg_table
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@reg_table
+POSTHOOK: Output: default@reg_table

Reply via email to