Repository: hive
Updated Branches:
  refs/heads/master fa24d4b9b -> 9d5d737db


http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java
new file mode 100644
index 0000000..5d7b9e5
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinBranchInfo.java
@@ -0,0 +1,45 @@
+/**
+ * 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.parse;
+
+
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+
+public class SemiJoinBranchInfo {
+  private TableScanOperator ts;
+  private boolean isHint;
+
+  public SemiJoinBranchInfo(TableScanOperator ts) {
+    this.ts = ts;
+    isHint = false;
+  }
+
+  public SemiJoinBranchInfo(TableScanOperator ts, boolean isHint) {
+    this.ts = ts;
+    this.isHint = isHint;
+  }
+
+  public TableScanOperator getTsOp() {
+    return ts;
+  }
+
+  public boolean getIsHint() {
+    return isHint;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinHint.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinHint.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinHint.java
new file mode 100644
index 0000000..1f24e23
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemiJoinHint.java
@@ -0,0 +1,43 @@
+/**
+ * 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.parse;
+
+public class SemiJoinHint {
+  private String tabAlias;
+  private String colName;
+  private Integer numEntries;
+
+  public SemiJoinHint(String tabAlias, String colName, Integer numEntries) {
+    this.tabAlias = tabAlias;
+    this.colName = colName;
+    this.numEntries = numEntries;
+  }
+
+  public String getTabAlias() {
+    return tabAlias;
+  }
+
+  public String getColName() {
+    return colName;
+  }
+
+  public Integer getNumEntries() {
+    return numEntries != null ? numEntries : -1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index 7caeb78..96525b4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -531,7 +531,7 @@ public abstract class TaskCompiler {
     clone.setLineageInfo(pCtx.getLineageInfo());
     clone.setMapJoinOps(pCtx.getMapJoinOps());
     clone.setRsToRuntimeValuesInfoMap(pCtx.getRsToRuntimeValuesInfoMap());
-    clone.setRsOpToTsOpMap(pCtx.getRsOpToTsOpMap());
+    clone.setRsToSemiJoinBranchInfo(pCtx.getRsToSemiJoinBranchInfo());
 
     return clone;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
----------------------------------------------------------------------
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 eaad988..26eda04 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
@@ -178,6 +178,9 @@ public class TezCompiler extends TaskCompiler {
     TableScanOperator victimTS = null;
     ReduceSinkOperator victimRS = null;
 
+    // If there is a hint and no operator is removed then throw error
+    boolean hasHint = false;
+    boolean removed = false;
     for (Operator<?> o : component) {
       // Look for AppMasterEventOperator or ReduceSinkOperator
       if (o instanceof AppMasterEventOperator) {
@@ -185,25 +188,34 @@ public class TezCompiler extends TaskCompiler {
                 || o.getStatistics().getDataSize() < victimAM.getStatistics()
                 .getDataSize()) {
           victimAM = (AppMasterEventOperator) o;
+          removed = true;
         }
       } else if (o instanceof ReduceSinkOperator) {
-        TableScanOperator ts = context.parseContext.getRsOpToTsOpMap().get(o);
-        if (ts == null) {
+
+        SemiJoinBranchInfo sjInfo =
+                context.parseContext.getRsToSemiJoinBranchInfo().get(o);
+        if (sjInfo == null ) continue;
+        if (sjInfo.getIsHint()) {
+          // Skipping because of hint. Mark this info,
+          hasHint = true;
           continue;
         }
+
+        TableScanOperator ts = sjInfo.getTsOp();
         // Sanity check
         assert component.contains(ts);
 
         if (victimRS == null ||
                 ts.getStatistics().getDataSize() <
-                victimTS.getStatistics().getDataSize()) {
-            victimRS = (ReduceSinkOperator) o;
-            victimTS = ts;
-          }
+                        victimTS.getStatistics().getDataSize()) {
+          victimRS = (ReduceSinkOperator) o;
+          victimTS = ts;
+          removed = true;
         }
       }
+    }
 
-    // Always set the min/max optimization as victim.
+    // Always set the semijoin optimization as victim.
     Operator<?> victim = victimRS;
 
     if (victimRS == null && victimAM != null ) {
@@ -227,6 +239,11 @@ public class TezCompiler extends TaskCompiler {
       }
     }
 
+    if (hasHint && !removed) {
+      // There is hint but none of the operators removed. Throw error
+      throw new SemanticException("The user hint is causing an operator cycle. 
Please fix it and retry");
+    }
+
     if (victim == null ||
             (!context.pruningOpsRemovedByPriorOpt.isEmpty() &&
                     context.pruningOpsRemovedByPriorOpt.contains(victim))) {
@@ -287,11 +304,12 @@ public class TezCompiler extends TaskCompiler {
       LOG.debug("Adding special edge: " + o.getName() + " --> " + 
ts.toString());
       children.add(ts);
     } else if (o instanceof ReduceSinkOperator){
-      // min/max case
+      // semijoin case
       children = new ArrayList<Operator<?>>();
       children.addAll(o.getChildOperators());
-      TableScanOperator ts = parseContext.getRsOpToTsOpMap().get(o);
-      if (ts != null) {
+      SemiJoinBranchInfo sjInfo = 
parseContext.getRsToSemiJoinBranchInfo().get(o);
+      if (sjInfo != null ) {
+        TableScanOperator ts = sjInfo.getTsOp();
         LOG.debug("Adding special edge: " + o.getName() + " --> " + 
ts.toString());
         children.add(ts);
       }
@@ -460,7 +478,7 @@ public class TezCompiler extends TaskCompiler {
     if (pCtx.getRsToRuntimeValuesInfoMap().size() > 0) {
       for (ReduceSinkOperator rs : 
pCtx.getRsToRuntimeValuesInfoMap().keySet()) {
         // Process min/max
-        GenTezUtils.processDynamicMinMaxPushDownOperator(
+        GenTezUtils.processDynamicSemiJoinPushDownOperator(
                 procCtx, pCtx.getRsToRuntimeValuesInfoMap().get(rs), rs);
       }
     }
@@ -617,7 +635,7 @@ public class TezCompiler extends TaskCompiler {
   private static void removeSemijoinOptimizationFromSMBJoins(
           OptimizeTezProcContext procCtx) throws SemanticException {
     if (!procCtx.conf.getBoolVar(ConfVars.TEZ_DYNAMIC_SEMIJOIN_REDUCTION) ||
-            procCtx.parseContext.getRsOpToTsOpMap().size() == 0) {
+            procCtx.parseContext.getRsToSemiJoinBranchInfo().size() == 0) {
       return;
     }
 
@@ -636,9 +654,9 @@ public class TezCompiler extends TaskCompiler {
     GraphWalker ogw = new PreOrderOnceWalker(disp);
     ogw.startWalking(topNodes, null);
 
+    List<TableScanOperator> tsOps = new ArrayList<>();
     // Iterate over the map and remove semijoin optimizations if needed.
     for (CommonMergeJoinOperator joinOp : ctx.JoinOpToTsOpMap.keySet()) {
-      List<TableScanOperator> tsOps = new ArrayList<TableScanOperator>();
       // Get one top level TS Op directly from the stack
       tsOps.add(ctx.JoinOpToTsOpMap.get(joinOp));
 
@@ -651,7 +669,7 @@ public class TezCompiler extends TaskCompiler {
         }
 
         assert parent instanceof SelectOperator;
-        while(parent != null) {
+        while (parent != null) {
           if (parent instanceof TableScanOperator) {
             tsOps.add((TableScanOperator) parent);
             break;
@@ -659,20 +677,24 @@ public class TezCompiler extends TaskCompiler {
           parent = parent.getParentOperators().get(0);
         }
       }
-
-      // Now the relevant TableScanOperators are known, find if there exists
-      // a semijoin filter on any of them, if so, remove it.
-      ParseContext pctx = procCtx.parseContext;
-      for (TableScanOperator ts : tsOps) {
-        for (ReduceSinkOperator rs : pctx.getRsOpToTsOpMap().keySet()) {
-          if (ts == pctx.getRsOpToTsOpMap().get(rs)) {
-            // match!
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Semijoin optimization found going to SMB join. 
Removing semijoin "
-                  + OperatorUtils.getOpNamePretty(rs) + " - " + 
OperatorUtils.getOpNamePretty(ts));
-            }
-            GenTezUtils.removeBranch(rs);
-            GenTezUtils.removeSemiJoinOperator(pctx, rs, ts);
+    }
+    // Now the relevant TableScanOperators are known, find if there exists
+    // a semijoin filter on any of them, if so, remove it.
+
+    ParseContext pctx = procCtx.parseContext;
+    for (TableScanOperator ts : tsOps) {
+      for (ReduceSinkOperator rs : pctx.getRsToSemiJoinBranchInfo().keySet()) {
+        SemiJoinBranchInfo sjInfo = pctx.getRsToSemiJoinBranchInfo().get(rs);
+        if (ts == sjInfo.getTsOp()) {
+          // match!
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Semijoin optimization found going to SMB join. Removing 
semijoin "
+                    + OperatorUtils.getOpNamePretty(rs) + " - " + 
OperatorUtils.getOpNamePretty(ts));
+          }
+          GenTezUtils.removeBranch(rs);
+          GenTezUtils.removeSemiJoinOperator(pctx, rs, ts);
+          if (sjInfo.getIsHint()) {
+            LOG.debug("Removing hinted semijoin as it is with SMB join " + rs 
+ " : " + ts);
           }
         }
       }
@@ -699,7 +721,7 @@ public class TezCompiler extends TaskCompiler {
   private static void removeSemiJoinCyclesDueToMapsideJoins(
           OptimizeTezProcContext procCtx) throws SemanticException {
     if (!procCtx.conf.getBoolVar(ConfVars.TEZ_DYNAMIC_SEMIJOIN_REDUCTION) ||
-            procCtx.parseContext.getRsOpToTsOpMap().size() == 0) {
+            procCtx.parseContext.getRsToSemiJoinBranchInfo().size() == 0) {
       return;
     }
 
@@ -752,10 +774,10 @@ public class TezCompiler extends TaskCompiler {
         }
 
         ReduceSinkOperator rs = ((ReduceSinkOperator) child);
-        TableScanOperator ts = pCtx.getRsOpToTsOpMap().get(rs);
-        if (ts == null) {
-          continue;
-        }
+        SemiJoinBranchInfo sjInfo = pCtx.getRsToSemiJoinBranchInfo().get(rs);
+        if (sjInfo == null) continue;
+
+        TableScanOperator ts = sjInfo.getTsOp();
         // This is a semijoin branch. Find if this is creating a potential
         // cycle with childJoin.
         for (Operator<?> parent : childJoin.getParentOperators()) {
@@ -776,6 +798,9 @@ public class TezCompiler extends TaskCompiler {
             }
             GenTezUtils.removeBranch(rs);
             GenTezUtils.removeSemiJoinOperator(pCtx, rs, ts);
+            if (sjInfo.getIsHint()) {
+              LOG.debug("Removing hinted semijoin as it is creating cycles 
with mapside joins " + rs + " : " + ts);
+            }
           }
         }
       }
@@ -790,8 +815,8 @@ public class TezCompiler extends TaskCompiler {
       assert nd instanceof ReduceSinkOperator;
       ReduceSinkOperator rs = (ReduceSinkOperator) nd;
       ParseContext pCtx = ((OptimizeTezProcContext) procCtx).parseContext;
-      TableScanOperator ts = pCtx.getRsOpToTsOpMap().get(rs);
-      if (ts == null) {
+      SemiJoinBranchInfo sjInfo = pCtx.getRsToSemiJoinBranchInfo().get(rs);
+      if (sjInfo == null) {
         // nothing to do here.
         return null;
       }
@@ -802,6 +827,7 @@ public class TezCompiler extends TaskCompiler {
       GroupByDesc gbDesc = gbOp.getConf();
       ArrayList<AggregationDesc> aggregationDescs = gbDesc.getAggregators();
       boolean removeSemiJoin = false;
+      TableScanOperator ts = sjInfo.getTsOp();
       for (AggregationDesc agg : aggregationDescs) {
         if (agg.getGenericUDAFName() != "bloom_filter") {
           continue;
@@ -809,20 +835,24 @@ public class TezCompiler extends TaskCompiler {
 
         GenericUDAFBloomFilterEvaluator udafBloomFilterEvaluator =
                 (GenericUDAFBloomFilterEvaluator) 
agg.getGenericUDAFEvaluator();
+        if (udafBloomFilterEvaluator.hasHintEntries())
+          return null; // Created using hint, skip it
+
         long expectedEntries = udafBloomFilterEvaluator.getExpectedEntries();
         if (expectedEntries == -1 || expectedEntries >
                 
pCtx.getConf().getLongVar(ConfVars.TEZ_MAX_BLOOM_FILTER_ENTRIES)) {
           removeSemiJoin = true;
           if (LOG.isDebugEnabled()) {
             LOG.debug("expectedEntries=" + expectedEntries + ". "
-                + "Either stats unavailable or expectedEntries exceeded max 
allowable bloomfilter size. "
-                + "Removing semijoin "
-                + OperatorUtils.getOpNamePretty(rs) + " - " + 
OperatorUtils.getOpNamePretty(ts));
+                    + "Either stats unavailable or expectedEntries exceeded 
max allowable bloomfilter size. "
+                    + "Removing semijoin "
+                    + OperatorUtils.getOpNamePretty(rs) + " - " + 
OperatorUtils.getOpNamePretty(ts));
           }
           break;
         }
       }
 
+      // At this point, hinted semijoin case has been handled already
       // Check if big table is big enough that runtime filtering is
       // worth it.
       if (ts.getStatistics() != null) {
@@ -831,16 +861,16 @@ public class TezCompiler extends TaskCompiler {
           removeSemiJoin = true;
           if (LOG.isDebugEnabled()) {
             LOG.debug("Insufficient rows (" + numRows + ") to justify semijoin 
optimization. Removing semijoin "
-                + OperatorUtils.getOpNamePretty(rs) + " - " + 
OperatorUtils.getOpNamePretty(ts));
+                    + OperatorUtils.getOpNamePretty(rs) + " - " + 
OperatorUtils.getOpNamePretty(ts));
           }
         }
       }
-
       if (removeSemiJoin) {
         // The stats are not annotated, remove the semijoin operator
         GenTezUtils.removeBranch(rs);
         GenTezUtils.removeSemiJoinOperator(pCtx, rs, ts);
       }
+
       return null;
     }
   }
@@ -905,15 +935,23 @@ public class TezCompiler extends TaskCompiler {
           }
 
           ReduceSinkOperator rs = (ReduceSinkOperator) child;
-          TableScanOperator ts = parseContext.getRsOpToTsOpMap().get(rs);
-          if (ts == null || ts != bigTableTS) {
-            // skip, no semijoin or not the one we are looking for.
+          SemiJoinBranchInfo sjInfo = 
parseContext.getRsToSemiJoinBranchInfo().get(rs);
+          if (sjInfo == null) continue;
+
+          TableScanOperator ts = sjInfo.getTsOp();
+          if (ts != bigTableTS) {
+            // skip, not the one we are looking for.
             continue;
           }
 
+          parallelEdges = true;
+
+          if (sjInfo.getIsHint()) {
+            // Created by hint, skip it
+            continue;
+          }
           // Add the semijoin branch to the map
           semijoins.put(rs, ts);
-          parallelEdges = true;
         }
       }
     }
@@ -1141,10 +1179,15 @@ public class TezCompiler extends TaskCompiler {
     }
 
     List<ReduceSinkOperator> semijoinRsToRemove = new 
ArrayList<ReduceSinkOperator>();
-    Map<ReduceSinkOperator, TableScanOperator> map = 
procCtx.parseContext.getRsOpToTsOpMap();
+    Map<ReduceSinkOperator, SemiJoinBranchInfo> map = 
procCtx.parseContext.getRsToSemiJoinBranchInfo();
     double semijoinReductionThreshold = procCtx.conf.getFloatVar(
         HiveConf.ConfVars.TEZ_DYNAMIC_SEMIJOIN_REDUCTION_THRESHOLD);
     for (ReduceSinkOperator rs : map.keySet()) {
+      SemiJoinBranchInfo sjInfo = map.get(rs);
+      if (sjInfo.getIsHint()) {
+        // Semijoin created using hint, skip it
+        continue;
+      }
       // rs is semijoin optimization branch, which should look like 
<Parent>-SEL-GB1-RS1-GB2-RS2
       // Get to the SelectOperator ancestor
       SelectOperator sel = null;
@@ -1159,7 +1202,7 @@ public class TezCompiler extends TaskCompiler {
       }
 
       // Check the ndv/rows from the SEL vs the destination tablescan the 
semijoin opt is going to.
-      TableScanOperator ts = map.get(rs);
+      TableScanOperator ts = sjInfo.getTsOp();
       RuntimeValuesInfo rti = 
procCtx.parseContext.getRsToRuntimeValuesInfoMap().get(rs);
       ExprNodeDesc tsExpr = rti.getTsColExpr();
       // In the SEL operator of the semijoin branch, there should be only one 
column in the operator
@@ -1179,7 +1222,7 @@ public class TezCompiler extends TaskCompiler {
     }
 
     for (ReduceSinkOperator rs : semijoinRsToRemove) {
-      TableScanOperator ts = map.get(rs);
+      TableScanOperator ts = map.get(rs).getTsOp();
       if (LOG.isDebugEnabled()) {
         LOG.debug("Reduction factor not satisfied for " + 
OperatorUtils.getOpNamePretty(rs)
             + "-" + OperatorUtils.getOpNamePretty(ts) + ". Removing semijoin 
optimization.");

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java
index 18e4fbd..3143554 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDynamicListDesc.java
@@ -18,7 +18,10 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
+import java.util.Map;
+
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.parse.SemiJoinHint;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 /**
@@ -29,14 +32,17 @@ public class ExprNodeDynamicListDesc extends ExprNodeDesc {
 
   Operator<? extends OperatorDesc> source;
   int keyIndex;
+  Map<String, SemiJoinHint> hints;
 
   public ExprNodeDynamicListDesc() {
   }
 
-  public ExprNodeDynamicListDesc(TypeInfo typeInfo, Operator<? extends 
OperatorDesc> source, int keyIndex) {
+  public ExprNodeDynamicListDesc(TypeInfo typeInfo, Operator<? extends 
OperatorDesc> source,
+      int keyIndex, Map<String, SemiJoinHint> hints) {
     super(typeInfo);
     this.source = source;
     this.keyIndex = keyIndex;
+    this.hints = hints;
   }
 
   public void setSource(Operator<? extends OperatorDesc> source) {
@@ -57,8 +63,7 @@ public class ExprNodeDynamicListDesc extends ExprNodeDesc {
 
   @Override
   public ExprNodeDesc clone() {
-    ExprNodeDynamicListDesc clone = new ExprNodeDynamicListDesc(typeInfo, 
source, keyIndex);
-    return clone;
+    return new ExprNodeDynamicListDesc(typeInfo, source, keyIndex, hints);
   }
 
   @Override
@@ -78,4 +83,8 @@ public class ExprNodeDynamicListDesc extends ExprNodeDesc {
   public String toString() {
     return source.toString();
   }
+
+  public Map<String, SemiJoinHint> getHints() {
+    return hints;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java 
b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
index bcf3691..032c7bb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
@@ -29,6 +29,7 @@ import java.util.Map;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.parse.QBJoinTree;
+import org.apache.hadoop.hive.ql.parse.SemiJoinHint;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
 
@@ -106,6 +107,10 @@ public class JoinDesc extends AbstractOperatorDesc {
   private transient Map<String, Operator<? extends OperatorDesc>> 
aliasToOpInfo;
   private transient boolean leftInputJoin;
   private transient List<String> streamAliases;
+  // Note: there are two things in Hive called semi-joins - the left semi join 
construct,
+  //       and also a bloom-filter based optimization that came later. This is 
for the latter.
+  //       Everything else in this desc that says "semi-join" is for the 
former.
+  private transient Map<String, SemiJoinHint> semiJoinHints;
 
   public JoinDesc() {
   }
@@ -197,6 +202,7 @@ public class JoinDesc extends AbstractOperatorDesc {
     this.filterMap = clone.filterMap;
     this.residualFilterExprs = clone.residualFilterExprs;
     this.statistics = clone.statistics;
+    this.semiJoinHints = clone.semiJoinHints;
   }
 
   public Map<Byte, List<ExprNodeDesc>> getExprs() {
@@ -682,4 +688,16 @@ public class JoinDesc extends AbstractOperatorDesc {
     streamAliases = joinDesc.streamAliases == null ? null : new 
ArrayList<String>(joinDesc.streamAliases);
   }
 
+  private static final org.slf4j.Logger LOG = 
org.slf4j.LoggerFactory.getLogger(JoinDesc.class);
+  public void setSemiJoinHints(Map<String, SemiJoinHint> semiJoinHints) {
+    if (semiJoinHints != null || this.semiJoinHints != null) {
+      LOG.debug("Setting semi-join hints to " + semiJoinHints);
+    }
+    this.semiJoinHints = semiJoinHints;
+  }
+
+  public Map<String, SemiJoinHint> getSemiJoinHints() {
+    return semiJoinHints;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java 
b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
index 71c7310..f45daa8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Stack;
 
+import org.apache.hadoop.hive.ql.parse.SemiJoinHint;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -134,13 +135,12 @@ public class SyntheticJoinPredicate extends Transform {
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
 
-      ParseContext pCtx = ((SyntheticContext) procCtx).getParseContext();
-
       @SuppressWarnings("unchecked")
       CommonJoinOperator<JoinDesc> join = (CommonJoinOperator<JoinDesc>) nd;
 
       ReduceSinkOperator source = (ReduceSinkOperator) stack.get(stack.size() 
- 2);
       int srcPos = join.getParentOperators().indexOf(source);
+      Map<String, SemiJoinHint> hints = join.getConf().getSemiJoinHints();
 
       List<Operator<? extends OperatorDesc>> parents = 
join.getParentOperators();
 
@@ -181,7 +181,7 @@ public class SyntheticJoinPredicate extends Transform {
           inArgs.add(sourceKeys.get(i));
 
           ExprNodeDynamicListDesc dynamicExpr =
-              new ExprNodeDynamicListDesc(targetKeys.get(i).getTypeInfo(), 
target, i);
+              new ExprNodeDynamicListDesc(targetKeys.get(i).getTypeInfo(), 
target, i, hints);
 
           inArgs.add(dynamicExpr);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBloomFilter.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBloomFilter.java 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBloomFilter.java
index 2b84beb..2413ae6 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBloomFilter.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFBloomFilter.java
@@ -72,6 +72,7 @@ public class GenericUDAFBloomFilter implements 
GenericUDAFResolver2 {
   public static class GenericUDAFBloomFilterEvaluator extends 
GenericUDAFEvaluator {
     // Source operator to get the number of entries
     private SelectOperator sourceOperator;
+    private long hintEntries = -1;
     private long maxEntries = 0;
     private long minEntries = 0;
     private float factor = 1;
@@ -254,6 +255,10 @@ public class GenericUDAFBloomFilter implements 
GenericUDAFResolver2 {
     }
 
     public long getExpectedEntries() {
+      // If hint is provided use that size.
+      if (hintEntries > 0 )
+        return hintEntries;
+
       long expectedEntries = -1;
       if (sourceOperator != null && sourceOperator.getStatistics() != null) {
         Statistics stats = sourceOperator.getStatistics();
@@ -294,6 +299,14 @@ public class GenericUDAFBloomFilter implements 
GenericUDAFResolver2 {
       this.sourceOperator = sourceOperator;
     }
 
+    public void setHintEntries(long hintEntries) {
+      this.hintEntries = hintEntries;
+    }
+
+    public boolean hasHintEntries() {
+      return hintEntries != -1;
+    }
+
     public void setMaxEntries(long maxEntries) {
       this.maxEntries = maxEntries;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/test/queries/clientpositive/semijoin_hint.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/semijoin_hint.q 
b/ql/src/test/queries/clientpositive/semijoin_hint.q
new file mode 100644
index 0000000..5de0c8c
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/semijoin_hint.q
@@ -0,0 +1,54 @@
+set hive.mapred.mode=nonstrict;
+set hive.explain.user=false;
+set hive.cbo.enable=true;
+set hive.compute.query.using.stats=false;
+set hive.mapred.mode=nonstrict;
+set hive.optimize.ppd=true;
+set hive.ppd.remove.duplicatefilters=true;
+set hive.tez.dynamic.partition.pruning=true;
+set hive.tez.dynamic.semijoin.reduction=true;
+set hive.optimize.metadataonly=false;
+set hive.optimize.index.filter=true;
+set hive.stats.autogather=true;
+set hive.tez.bigtable.minsize.semijoin.reduction=1;
+set hive.tez.min.bloom.filter.entries=1;
+set hive.tez.dynamic.semijoin.reduction.threshold=-999999999999;
+
+-- Create Tables
+create table alltypesorc_int ( cint int, cstring string ) stored as ORC;
+create table srcpart_date (str string, value string) partitioned by (ds string 
) stored as ORC;
+CREATE TABLE srcpart_small(key1 STRING, value1 STRING) partitioned by (ds 
string) STORED as ORC;
+
+-- Add Partitions
+alter table srcpart_date add partition (ds = "2008-04-08");
+alter table srcpart_date add partition (ds = "2008-04-09");
+
+alter table srcpart_small add partition (ds = "2008-04-08");
+alter table srcpart_small add partition (ds = "2008-04-09");
+
+-- Load
+insert overwrite table alltypesorc_int select cint, cstring1 from alltypesorc;
+insert overwrite table srcpart_date partition (ds = "2008-04-08" ) select key, 
value from srcpart where ds = "2008-04-08";
+insert overwrite table srcpart_date partition (ds = "2008-04-09") select key, 
value from srcpart where ds = "2008-04-09";
+insert overwrite table srcpart_small partition (ds = "2008-04-09") select key, 
value from srcpart where ds = "2008-04-09";
+analyze table alltypesorc_int compute statistics for columns;
+analyze table srcpart_date compute statistics for columns;
+analyze table srcpart_small compute statistics for columns;
+
+set hive.cbo.returnpath.hiveop=true;
+
+create table srccc as select * from src;
+
+EXPLAIN select  /*+ semi(k, str, 5000)*/ count(*) from srcpart_date k join 
srcpart_small v on (k.str = v.key1) join alltypesorc_int i on (k.value = 
i.cstring);
+EXPLAIN select  /*+ semi(i, 3000)*/ count(*) from srcpart_date k join 
srcpart_small v on (k.str = v.key1) join alltypesorc_int i on (v.key1 = 
i.cstring);
+
+explain select /*+ semi(k, str, 1000)*/ count(*) from srcpart_date k join 
srcpart_small v on (k.str = v.key1);
+
+set hive.cbo.returnpath.hiveop=false;
+
+explain select /*+ semi(k, 1000)*/ count(*) from srcpart_date k join 
srcpart_small v on (k.str = v.key1);
+
+set hive.cbo.enable=false;
+
+explain select /*+ semi(k, str, 1000)*/ count(*) from srcpart_date k join 
srcpart_small v on (k.str = v.key1);
+

http://git-wip-us.apache.org/repos/asf/hive/blob/9d5d737d/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out 
b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
new file mode 100644
index 0000000..bac9240
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/semijoin_hint.q.out
@@ -0,0 +1,899 @@
+PREHOOK: query: create table alltypesorc_int ( cint int, cstring string ) 
stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@alltypesorc_int
+POSTHOOK: query: create table alltypesorc_int ( cint int, cstring string ) 
stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@alltypesorc_int
+PREHOOK: query: create table srcpart_date (str string, value string) 
partitioned by (ds string ) stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcpart_date
+POSTHOOK: query: create table srcpart_date (str string, value string) 
partitioned by (ds string ) stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcpart_date
+PREHOOK: query: CREATE TABLE srcpart_small(key1 STRING, value1 STRING) 
partitioned by (ds string) STORED as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcpart_small
+POSTHOOK: query: CREATE TABLE srcpart_small(key1 STRING, value1 STRING) 
partitioned by (ds string) STORED as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcpart_small
+PREHOOK: query: alter table srcpart_date add partition (ds = "2008-04-08")
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@srcpart_date
+POSTHOOK: query: alter table srcpart_date add partition (ds = "2008-04-08")
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@srcpart_date
+POSTHOOK: Output: default@srcpart_date@ds=2008-04-08
+PREHOOK: query: alter table srcpart_date add partition (ds = "2008-04-09")
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@srcpart_date
+POSTHOOK: query: alter table srcpart_date add partition (ds = "2008-04-09")
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@srcpart_date
+POSTHOOK: Output: default@srcpart_date@ds=2008-04-09
+PREHOOK: query: alter table srcpart_small add partition (ds = "2008-04-08")
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@srcpart_small
+POSTHOOK: query: alter table srcpart_small add partition (ds = "2008-04-08")
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@srcpart_small
+POSTHOOK: Output: default@srcpart_small@ds=2008-04-08
+PREHOOK: query: alter table srcpart_small add partition (ds = "2008-04-09")
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@srcpart_small
+POSTHOOK: query: alter table srcpart_small add partition (ds = "2008-04-09")
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@srcpart_small
+POSTHOOK: Output: default@srcpart_small@ds=2008-04-09
+PREHOOK: query: insert overwrite table alltypesorc_int select cint, cstring1 
from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@alltypesorc_int
+POSTHOOK: query: insert overwrite table alltypesorc_int select cint, cstring1 
from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@alltypesorc_int
+POSTHOOK: Lineage: alltypesorc_int.cint SIMPLE 
[(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: alltypesorc_int.cstring SIMPLE 
[(alltypesorc)alltypesorc.FieldSchema(name:cstring1, type:string, 
comment:null), ]
+PREHOOK: query: insert overwrite table srcpart_date partition (ds = 
"2008-04-08" ) select key, value from srcpart where ds = "2008-04-08"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+PREHOOK: Output: default@srcpart_date@ds=2008-04-08
+POSTHOOK: query: insert overwrite table srcpart_date partition (ds = 
"2008-04-08" ) select key, value from srcpart where ds = "2008-04-08"
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
+POSTHOOK: Output: default@srcpart_date@ds=2008-04-08
+POSTHOOK: Lineage: srcpart_date PARTITION(ds=2008-04-08).str SIMPLE 
[(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: srcpart_date PARTITION(ds=2008-04-08).value SIMPLE 
[(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: insert overwrite table srcpart_date partition (ds = 
"2008-04-09") select key, value from srcpart where ds = "2008-04-09"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Output: default@srcpart_date@ds=2008-04-09
+POSTHOOK: query: insert overwrite table srcpart_date partition (ds = 
"2008-04-09") select key, value from srcpart where ds = "2008-04-09"
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: default@srcpart_date@ds=2008-04-09
+POSTHOOK: Lineage: srcpart_date PARTITION(ds=2008-04-09).str SIMPLE 
[(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: srcpart_date PARTITION(ds=2008-04-09).value SIMPLE 
[(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: insert overwrite table srcpart_small partition (ds = 
"2008-04-09") select key, value from srcpart where ds = "2008-04-09"
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+PREHOOK: Output: default@srcpart_small@ds=2008-04-09
+POSTHOOK: query: insert overwrite table srcpart_small partition (ds = 
"2008-04-09") select key, value from srcpart where ds = "2008-04-09"
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
+POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
+POSTHOOK: Output: default@srcpart_small@ds=2008-04-09
+POSTHOOK: Lineage: srcpart_small PARTITION(ds=2008-04-09).key1 SIMPLE 
[(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: srcpart_small PARTITION(ds=2008-04-09).value1 SIMPLE 
[(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: analyze table alltypesorc_int compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc_int
+PREHOOK: Output: default@alltypesorc_int
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table alltypesorc_int compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc_int
+POSTHOOK: Output: default@alltypesorc_int
+#### A masked pattern was here ####
+PREHOOK: query: analyze table srcpart_date compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart_date
+PREHOOK: Input: default@srcpart_date@ds=2008-04-08
+PREHOOK: Input: default@srcpart_date@ds=2008-04-09
+PREHOOK: Output: default@srcpart_date
+PREHOOK: Output: default@srcpart_date@ds=2008-04-08
+PREHOOK: Output: default@srcpart_date@ds=2008-04-09
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table srcpart_date compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart_date
+POSTHOOK: Input: default@srcpart_date@ds=2008-04-08
+POSTHOOK: Input: default@srcpart_date@ds=2008-04-09
+POSTHOOK: Output: default@srcpart_date
+POSTHOOK: Output: default@srcpart_date@ds=2008-04-08
+POSTHOOK: Output: default@srcpart_date@ds=2008-04-09
+#### A masked pattern was here ####
+PREHOOK: query: analyze table srcpart_small compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@srcpart_small
+PREHOOK: Input: default@srcpart_small@ds=2008-04-08
+PREHOOK: Input: default@srcpart_small@ds=2008-04-09
+PREHOOK: Output: default@srcpart_small
+PREHOOK: Output: default@srcpart_small@ds=2008-04-08
+PREHOOK: Output: default@srcpart_small@ds=2008-04-09
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table srcpart_small compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@srcpart_small
+POSTHOOK: Input: default@srcpart_small@ds=2008-04-08
+POSTHOOK: Input: default@srcpart_small@ds=2008-04-09
+POSTHOOK: Output: default@srcpart_small
+POSTHOOK: Output: default@srcpart_small@ds=2008-04-08
+POSTHOOK: Output: default@srcpart_small@ds=2008-04-09
+#### A masked pattern was here ####
+PREHOOK: query: create table srccc as select * from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srccc
+POSTHOOK: query: create table srccc as select * from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srccc
+POSTHOOK: Lineage: srccc.key SIMPLE [(src)src.FieldSchema(name:key, 
type:string, comment:default), ]
+POSTHOOK: Lineage: srccc.value SIMPLE [(src)src.FieldSchema(name:value, 
type:string, comment:default), ]
+PREHOOK: query: EXPLAIN select  /*+ semi(k, str, 5000)*/ count(*) from 
srcpart_date k join srcpart_small v on (k.str = v.key1) join alltypesorc_int i 
on (k.value = i.cstring)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select  /*+ semi(k, str, 5000)*/ count(*) from 
srcpart_date k join srcpart_small v on (k.str = v.key1) join alltypesorc_int i 
on (k.value = i.cstring)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Reducer 7 (BROADCAST_EDGE)
+        Map 8 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+        Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 7 <- Map 6 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: i
+                  filterExpr: (cstring is not null and (cstring BETWEEN 
DynamicValue(RS_7_k_cstring_min) AND DynamicValue(RS_7_k_cstring_max) and 
in_bloom_filter(cstring, DynamicValue(RS_7_k_cstring_bloom_filter)))) (type: 
boolean)
+                  Statistics: Num rows: 12288 Data size: 862450 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (cstring is not null and (cstring BETWEEN 
DynamicValue(RS_7_k_cstring_min) AND DynamicValue(RS_7_k_cstring_max) and 
in_bloom_filter(cstring, DynamicValue(RS_7_k_cstring_bloom_filter)))) (type: 
boolean)
+                    Statistics: Num rows: 9174 Data size: 643900 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cstring (type: string)
+                      outputColumnNames: cstring
+                      Statistics: Num rows: 9174 Data size: 643900 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: cstring (type: string)
+                        sort order: +
+                        Map-reduce partition columns: cstring (type: string)
+                        Statistics: Num rows: 9174 Data size: 643900 Basic 
stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: k
+                  filterExpr: (str is not null and value is not null) (type: 
boolean)
+                  Statistics: Num rows: 2000 Data size: 356000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (str is not null and value is not null) (type: 
boolean)
+                    Statistics: Num rows: 2000 Data size: 356000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: str (type: string), value (type: string)
+                      outputColumnNames: str, value
+                      Statistics: Num rows: 2000 Data size: 356000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: value (type: string)
+                        sort order: +
+                        Map-reduce partition columns: value (type: string)
+                        Statistics: Num rows: 2000 Data size: 356000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                        value expressions: str (type: string)
+                      Select Operator
+                        expressions: value (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 2000 Data size: 182000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), 
bloom_filter(_col0, expectedEntries=5000)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic 
stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 
(type: string), _col2 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 8 
+            Map Operator Tree:
+                TableScan
+                  alias: v
+                  filterExpr: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_9_i_key1_min) AND DynamicValue(RS_9_i_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_9_i_key1_bloom_filter)))) (type: boolean)
+                  Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                  Filter Operator
+                    predicate: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_9_i_key1_min) AND DynamicValue(RS_9_i_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_9_i_key1_bloom_filter)))) (type: boolean)
+                    Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                    Select Operator
+                      expressions: key1 (type: string)
+                      outputColumnNames: key1
+                      Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                      Reduce Output Operator
+                        key expressions: key1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: key1 (type: string)
+                        Statistics: Num rows: 1000 Data size: 87000 Basic 
stats: COMPLETE Column stats: PARTIAL
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 cstring (type: string)
+                  1 value (type: string)
+                outputColumnNames: str
+                Statistics: Num rows: 3281 Data size: 285447 Basic stats: 
COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: str (type: string)
+                  sort order: +
+                  Map-reduce partition columns: str (type: string)
+                  Statistics: Num rows: 3281 Data size: 285447 Basic stats: 
COMPLETE Column stats: COMPLETE
+                Select Operator
+                  expressions: str (type: string)
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 3281 Data size: 285447 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Group By Operator
+                    aggregations: min(_col0), max(_col0), bloom_filter(_col0, 
expectedEntries=410)
+                    mode: hash
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: string), _col1 (type: 
string), _col2 (type: binary)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 str (type: string)
+                  1 key1 (type: string)
+                Statistics: Num rows: 16004 Data size: 128032 Basic stats: 
COMPLETE Column stats: PARTIAL
+                Select Operator
+                  Statistics: Num rows: 16004 Data size: 64016 Basic stats: 
COMPLETE Column stats: PARTIAL
+                  Group By Operator
+                    aggregations: count()
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: 
COMPLETE Column stats: PARTIAL
+                      value expressions: _col0 (type: bigint)
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: $f0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                  table:
+                      input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), 
bloom_filter(VALUE._col2, expectedEntries=410)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: 
string), _col2 (type: binary)
+        Reducer 7 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), 
bloom_filter(VALUE._col2, expectedEntries=5000)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: 
string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: EXPLAIN select  /*+ semi(i, 3000)*/ count(*) from srcpart_date 
k join srcpart_small v on (k.str = v.key1) join alltypesorc_int i on (v.key1 = 
i.cstring)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN select  /*+ semi(i, 3000)*/ count(*) from 
srcpart_date k join srcpart_small v on (k.str = v.key1) join alltypesorc_int i 
on (v.key1 = i.cstring)
+POSTHOOK: type: QUERY
+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 6 <- Reducer 4 (BROADCAST_EDGE)
+        Map 7 <- Reducer 5 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE), Map 7 
(SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+        Reducer 5 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: i
+                  filterExpr: cstring is not null (type: boolean)
+                  Statistics: Num rows: 12288 Data size: 862450 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: cstring is not null (type: boolean)
+                    Statistics: Num rows: 9174 Data size: 643900 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: cstring (type: string)
+                      outputColumnNames: cstring
+                      Statistics: Num rows: 9174 Data size: 643900 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: cstring (type: string)
+                        sort order: +
+                        Map-reduce partition columns: cstring (type: string)
+                        Statistics: Num rows: 9174 Data size: 643900 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: cstring (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 9174 Data size: 643900 Basic 
stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), 
bloom_filter(_col0, expectedEntries=3000)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic 
stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 
(type: string), _col2 (type: binary)
+                      Select Operator
+                        expressions: cstring (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 9174 Data size: 643900 Basic 
stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), 
bloom_filter(_col0, expectedEntries=3000)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic 
stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 
(type: string), _col2 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: v
+                  filterExpr: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_3_i_key1_min) AND DynamicValue(RS_3_i_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_3_i_key1_bloom_filter)))) (type: boolean)
+                  Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                  Filter Operator
+                    predicate: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_3_i_key1_min) AND DynamicValue(RS_3_i_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_3_i_key1_bloom_filter)))) (type: boolean)
+                    Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                    Select Operator
+                      expressions: key1 (type: string)
+                      outputColumnNames: key1
+                      Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                      Reduce Output Operator
+                        key expressions: key1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: key1 (type: string)
+                        Statistics: Num rows: 1000 Data size: 87000 Basic 
stats: COMPLETE Column stats: PARTIAL
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: k
+                  filterExpr: (str is not null and (str BETWEEN 
DynamicValue(RS_3_i_str_min) AND DynamicValue(RS_3_i_str_max) and 
in_bloom_filter(str, DynamicValue(RS_3_i_str_bloom_filter)))) (type: boolean)
+                  Statistics: Num rows: 2000 Data size: 174000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (str is not null and (str BETWEEN 
DynamicValue(RS_3_i_str_min) AND DynamicValue(RS_3_i_str_max) and 
in_bloom_filter(str, DynamicValue(RS_3_i_str_bloom_filter)))) (type: boolean)
+                    Statistics: Num rows: 2000 Data size: 174000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: str (type: string)
+                      outputColumnNames: str
+                      Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: str (type: string)
+                        sort order: +
+                        Map-reduce partition columns: str (type: string)
+                        Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                     Inner Join 1 to 2
+                keys:
+                  0 cstring (type: string)
+                  1 key1 (type: string)
+                  2 str (type: string)
+                Statistics: Num rows: 16008 Data size: 128064 Basic stats: 
COMPLETE Column stats: PARTIAL
+                Select Operator
+                  Statistics: Num rows: 16008 Data size: 64032 Basic stats: 
COMPLETE Column stats: PARTIAL
+                  Group By Operator
+                    aggregations: count()
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: 
COMPLETE Column stats: PARTIAL
+                      value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: $f0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                  table:
+                      input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), 
bloom_filter(VALUE._col2, expectedEntries=3000)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: 
string), _col2 (type: binary)
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), 
bloom_filter(VALUE._col2, expectedEntries=3000)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: 
string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select /*+ semi(k, str, 1000)*/ count(*) from 
srcpart_date k join srcpart_small v on (k.str = v.key1)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select /*+ semi(k, str, 1000)*/ count(*) from 
srcpart_date k join srcpart_small v on (k.str = v.key1)
+POSTHOOK: type: QUERY
+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 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: k
+                  filterExpr: str is not null (type: boolean)
+                  Statistics: Num rows: 2000 Data size: 174000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: str is not null (type: boolean)
+                    Statistics: Num rows: 2000 Data size: 174000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: str (type: string)
+                      outputColumnNames: str
+                      Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: str (type: string)
+                        sort order: +
+                        Map-reduce partition columns: str (type: string)
+                        Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: str (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), 
bloom_filter(_col0, expectedEntries=1000)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic 
stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 
(type: string), _col2 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: v
+                  filterExpr: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_3_k_key1_min) AND DynamicValue(RS_3_k_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_3_k_key1_bloom_filter)))) (type: boolean)
+                  Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                  Filter Operator
+                    predicate: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_3_k_key1_min) AND DynamicValue(RS_3_k_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_3_k_key1_bloom_filter)))) (type: boolean)
+                    Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                    Select Operator
+                      expressions: key1 (type: string)
+                      outputColumnNames: key1
+                      Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                      Reduce Output Operator
+                        key expressions: key1 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: key1 (type: string)
+                        Statistics: Num rows: 1000 Data size: 87000 Basic 
stats: COMPLETE Column stats: PARTIAL
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 str (type: string)
+                  1 key1 (type: string)
+                Statistics: Num rows: 9756 Data size: 78048 Basic stats: 
COMPLETE Column stats: PARTIAL
+                Select Operator
+                  Statistics: Num rows: 9756 Data size: 39024 Basic stats: 
COMPLETE Column stats: PARTIAL
+                  Group By Operator
+                    aggregations: count()
+                    mode: hash
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: 
COMPLETE Column stats: PARTIAL
+                      value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: $f0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                  table:
+                      input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), 
bloom_filter(VALUE._col2, expectedEntries=1000)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: 
string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select /*+ semi(k, 1000)*/ count(*) from srcpart_date 
k join srcpart_small v on (k.str = v.key1)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select /*+ semi(k, 1000)*/ count(*) from srcpart_date 
k join srcpart_small v on (k.str = v.key1)
+POSTHOOK: type: QUERY
+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 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: k
+                  filterExpr: str is not null (type: boolean)
+                  Statistics: Num rows: 2000 Data size: 174000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: str is not null (type: boolean)
+                    Statistics: Num rows: 2000 Data size: 174000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: str (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Select Operator
+                        expressions: _col0 (type: string)
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                        Group By Operator
+                          aggregations: min(_col0), max(_col0), 
bloom_filter(_col0, expectedEntries=1000)
+                          mode: hash
+                          outputColumnNames: _col0, _col1, _col2
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                          Reduce Output Operator
+                            sort order: 
+                            Statistics: Num rows: 1 Data size: 552 Basic 
stats: COMPLETE Column stats: COMPLETE
+                            value expressions: _col0 (type: string), _col1 
(type: string), _col2 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: v
+                  filterExpr: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_6_k_key1_min) AND DynamicValue(RS_6_k_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_6_k_key1_bloom_filter)))) (type: boolean)
+                  Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                  Filter Operator
+                    predicate: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_6_k_key1_min) AND DynamicValue(RS_6_k_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_6_k_key1_bloom_filter)))) (type: boolean)
+                    Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                    Select Operator
+                      expressions: key1 (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                      Reduce Output Operator
+                        key expressions: _col0 (type: string)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: string)
+                        Statistics: Num rows: 1000 Data size: 87000 Basic 
stats: COMPLETE Column stats: PARTIAL
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: string)
+                  1 _col0 (type: string)
+                Statistics: Num rows: 9756 Data size: 78048 Basic stats: 
COMPLETE Column stats: PARTIAL
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                  table:
+                      input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), 
bloom_filter(VALUE._col2, expectedEntries=1000)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: 
string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain select /*+ semi(k, str, 1000)*/ count(*) from 
srcpart_date k join srcpart_small v on (k.str = v.key1)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select /*+ semi(k, str, 1000)*/ count(*) from 
srcpart_date k join srcpart_small v on (k.str = v.key1)
+POSTHOOK: type: QUERY
+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 5 <- Reducer 4 (BROADCAST_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE)
+        Reducer 4 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: k
+                  filterExpr: str is not null (type: boolean)
+                  Statistics: Num rows: 2000 Data size: 174000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: str is not null (type: boolean)
+                    Statistics: Num rows: 2000 Data size: 174000 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: str (type: string)
+                      sort order: +
+                      Map-reduce partition columns: str (type: string)
+                      Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: str (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 2000 Data size: 174000 Basic 
stats: COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        aggregations: min(_col0), max(_col0), 
bloom_filter(_col0, expectedEntries=1000)
+                        mode: hash
+                        outputColumnNames: _col0, _col1, _col2
+                        Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          sort order: 
+                          Statistics: Num rows: 1 Data size: 552 Basic stats: 
COMPLETE Column stats: COMPLETE
+                          value expressions: _col0 (type: string), _col1 
(type: string), _col2 (type: binary)
+            Execution mode: llap
+            LLAP IO: all inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: v
+                  filterExpr: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_3_k_key1_min) AND DynamicValue(RS_3_k_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_3_k_key1_bloom_filter)))) (type: boolean)
+                  Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                  Filter Operator
+                    predicate: (key1 is not null and (key1 BETWEEN 
DynamicValue(RS_3_k_key1_min) AND DynamicValue(RS_3_k_key1_max) and 
in_bloom_filter(key1, DynamicValue(RS_3_k_key1_bloom_filter)))) (type: boolean)
+                    Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+                    Reduce Output Operator
+                      key expressions: key1 (type: string)
+                      sort order: +
+                      Map-reduce partition columns: key1 (type: string)
+                      Statistics: Num rows: 1000 Data size: 87000 Basic stats: 
COMPLETE Column stats: PARTIAL
+            Execution mode: llap
+            LLAP IO: all inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 str (type: string)
+                  1 key1 (type: string)
+                Statistics: Num rows: 9756 Data size: 78048 Basic stats: 
COMPLETE Column stats: PARTIAL
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: PARTIAL
+                  table:
+                      input format: 
org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: 
org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), 
bloom_filter(VALUE._col2, expectedEntries=1000)
+                mode: final
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 552 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col0 (type: string), _col1 (type: 
string), _col2 (type: binary)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+

Reply via email to