Author: xuefu
Date: Thu Dec 18 19:58:10 2014
New Revision: 1646523
URL: http://svn.apache.org/r1646523
Log:
HIVE-8639: Convert SMBJoin to MapJoin [Spark Branch] (Szehon via Xuefu)
Added:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java
Modified:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join32.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join_stats.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_13.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_14.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_15.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_2.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_4.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_6.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_7.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_8.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/cross_product_check_2.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/parquet_join.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out
Modified:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
(original)
+++
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
Thu Dec 18 19:58:10 2014
@@ -113,7 +113,7 @@ public class Optimizer {
// If optimize hive.optimize.bucketmapjoin.sortedmerge is set, add both
// BucketMapJoinOptimizer and SortedMergeBucketMapJoinOptimizer
if ((HiveConf.getBoolVar(hiveConf,
HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN))
- && !isTezExecEngine) {
+ && !isTezExecEngine && !isSparkExecEngine) {
if (!bucketMapJoinOptimizer) {
// No need to add BucketMapJoinOptimizer twice
transformations.add(new BucketMapJoinOptimizer());
Added:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java?rev=1646523&view=auto
==============================================================================
---
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java
(added)
+++
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java
Thu Dec 18 19:58:10 2014
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.optimizer.spark;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext;
+
+import java.util.Stack;
+
+/**
+ * Converts a join to a more optimized join for the Spark path. Delegates to
a more specialized join processor.
+ */
+public class SparkJoinOptimizer implements NodeProcessor {
+
+ private SparkSortMergeJoinOptimizer smbJoinOptimizer;
+ private SparkMapJoinOptimizer mapJoinOptimizer;
+
+ public SparkJoinOptimizer(ParseContext procCtx) {
+ smbJoinOptimizer = new SparkSortMergeJoinOptimizer(procCtx);
+ mapJoinOptimizer = new SparkMapJoinOptimizer();
+ }
+
+ @Override
+ public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
Object... nodeOutputs) throws SemanticException {
+ OptimizeSparkProcContext context = (OptimizeSparkProcContext) procCtx;
+ HiveConf conf = context.getConf();
+
+ if (conf.getBoolVar(HiveConf.ConfVars.HIVE_AUTO_SORTMERGE_JOIN_TOMAPJOIN))
{
+ Object mapJoinOp = mapJoinOptimizer.process(nd, stack, procCtx,
nodeOutputs);
+ if (mapJoinOp == null) {
+ smbJoinOptimizer.process(nd, stack, procCtx, nodeOutputs);
+ }
+ } else {
+ Object sortMergeJoinOp = smbJoinOptimizer.process(nd, stack, procCtx,
nodeOutputs);
+ if (sortMergeJoinOp == null) {
+ mapJoinOptimizer.process(nd, stack, procCtx, nodeOutputs);
+ }
+ }
+ return null;
+ }
+}
Modified:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
(original)
+++
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
Thu Dec 18 19:58:10 2014
@@ -163,7 +163,7 @@ public class SparkMapJoinOptimizer imple
context.getMjOpSizes().put(mapJoinOp, mapJoinInfo[1] + mapJoinInfo[2]);
- return null;
+ return mapJoinOp;
}
// replaces the join operator with a new CommonJoinOperator, removes the
Modified:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
(original)
+++
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
Thu Dec 18 19:58:10 2014
@@ -18,13 +18,7 @@
package org.apache.hadoop.hive.ql.optimizer.spark;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Stack;
-
+import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hive.conf.HiveConf;
@@ -53,13 +47,17 @@ import org.apache.hadoop.hive.ql.plan.Ha
import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
import org.apache.hadoop.hive.ql.plan.PlanUtils;
-import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty;
import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc;
import org.apache.hadoop.hive.ql.plan.SparkWork;
import org.apache.hadoop.hive.ql.plan.TableDesc;
-import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
public class SparkReduceSinkMapJoinProc implements NodeProcessor {
@@ -185,19 +183,6 @@ public class SparkReduceSinkMapJoinProc
SparkWork sparkWork = context.currentTask.getWork();
LOG.debug("connecting "+parentWork.getName()+" with
"+myWork.getName());
sparkWork.connect(parentWork, myWork, edgeProp);
-
- ReduceSinkOperator r;
- if (parentRS.getConf().getOutputName() != null) {
- LOG.debug("Cloning reduce sink for multi-child broadcast edge");
- // we've already set this one up. Need to clone for the next work.
- r = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(
- (ReduceSinkDesc) parentRS.getConf().clone(),
parentRS.getParentOperators());
- context.clonedReduceSinks.add(r);
- } else {
- r = parentRS;
- }
- // remember the output name of the reduce sink
- r.getConf().setOutputName(myWork.getName());
}
}
Added:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java?rev=1646523&view=auto
==============================================================================
---
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java
(added)
+++
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java
Thu Dec 18 19:58:10 2014
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.optimizer.spark;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+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.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessor;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.optimizer.AbstractSMBJoinProc;
+import org.apache.hadoop.hive.ql.optimizer.SortBucketJoinProcCtx;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+
+import java.util.Stack;
+
+/**
+ * Converts a join operator to an SMB join if eligible.
+ */
+public class SparkSortMergeJoinOptimizer extends AbstractSMBJoinProc
implements NodeProcessor {
+
+ public SparkSortMergeJoinOptimizer(ParseContext pctx) {
+ super(pctx);
+ }
+
+ public SparkSortMergeJoinOptimizer() {
+ }
+
+ @Override
+ public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+
+ JoinOperator joinOp = (JoinOperator) nd;
+ HiveConf conf = ((OptimizeSparkProcContext)
procCtx).getParseContext().getConf();
+
+ if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_AUTO_SORTMERGE_JOIN)) {
+ return null;
+ }
+
+ SortBucketJoinProcCtx smbJoinContext = new SortBucketJoinProcCtx(conf);
+
+ boolean convert =
+ canConvertJoinToSMBJoin(
+ joinOp, smbJoinContext, pGraphContext, stack);
+
+ if (convert) {
+ return convertJoinToSMBJoinAndReturn(joinOp, smbJoinContext,
pGraphContext);
+ }
+ return null;
+ }
+
+ protected boolean canConvertJoinToSMBJoin(JoinOperator joinOperator,
SortBucketJoinProcCtx smbJoinContext,
+ ParseContext pGraphContext, Stack<Node> stack) throws SemanticException {
+ if (!supportBucketMapJoin(stack)) {
+ return false;
+ }
+ return canConvertJoinToSMBJoin(joinOperator, smbJoinContext,
pGraphContext);
+ }
+
+ //Preliminary checks. In the MR version of the code, these used to be done
via another walk, refactoring to be inline.
+ private boolean supportBucketMapJoin(Stack<Node> stack) {
+ int size = stack.size();
+ if (!(stack.get(size-1) instanceof JoinOperator) ||
+ !(stack.get(size-2) instanceof ReduceSinkOperator)) {
+ return false;
+ }
+
+ // If any operator in the stack does not support a auto-conversion, this
join should
+ // not be converted.
+ for (int pos = size -3; pos >= 0; pos--) {
+ Operator<? extends OperatorDesc> op = (Operator<? extends
OperatorDesc>)stack.get(pos);
+ if (!op.supportAutomaticSortMergeJoin()) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ protected SMBMapJoinOperator convertJoinToSMBJoinAndReturn(
+ JoinOperator joinOp,
+ SortBucketJoinProcCtx smbJoinContext,
+ ParseContext parseContext) throws SemanticException {
+ MapJoinOperator mapJoinOp = convertJoinToBucketMapJoin(joinOp,
smbJoinContext, parseContext);
+ SMBMapJoinOperator smbMapJoinOp =
+ convertBucketMapJoinToSMBJoin(mapJoinOp, smbJoinContext,
parseContext);
+ smbMapJoinOp.setConvertedAutomaticallySMBJoin(true);
+ return smbMapJoinOp;
+ }
+}
Modified:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
(original)
+++
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
Thu Dec 18 19:58:10 2014
@@ -100,24 +100,6 @@ public class GenSparkWork implements Nod
}
SparkWork sparkWork = context.currentTask.getWork();
-
- if (GenSparkUtils.getChildOperator(root, DummyStoreOperator.class) !=
null) {
- /*
- * SMB join case:
- *
- * (Big) (Small) (Small)
- * TS TS TS
- * \ | /
- * \ DS DS
- * \ | /
- * SMBJoinOP
- *
- * Only create MapWork rooted at TS of big table.
- * If there are dummy-store operators anywhere in TS's children path,
then this is for the small tables.
- * No separate Map-Task need to be created for small table TS, as they
will be read by the MapWork of the big-table.
- */
- return null;
- }
SMBMapJoinOperator smbOp = GenSparkUtils.getChildOperator(root,
SMBMapJoinOperator.class);
// Right now the work graph is pretty simple. If there is no
Modified:
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
(original)
+++
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
Thu Dec 18 19:58:10 2014
@@ -33,6 +33,7 @@ import org.apache.commons.logging.LogFac
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.Context;
import org.apache.hadoop.hive.ql.exec.ConditionalTask;
+import org.apache.hadoop.hive.ql.exec.DummyStoreOperator;
import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
import org.apache.hadoop.hive.ql.exec.JoinOperator;
import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
@@ -46,9 +47,9 @@ import org.apache.hadoop.hive.ql.exec.sp
import org.apache.hadoop.hive.ql.hooks.ReadEntity;
import org.apache.hadoop.hive.ql.hooks.WriteEntity;
import org.apache.hadoop.hive.ql.lib.CompositeProcessor;
+import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
import org.apache.hadoop.hive.ql.lib.Dispatcher;
-import org.apache.hadoop.hive.ql.lib.ForwardWalker;
import org.apache.hadoop.hive.ql.lib.GraphWalker;
import org.apache.hadoop.hive.ql.lib.Node;
import org.apache.hadoop.hive.ql.lib.NodeProcessor;
@@ -66,7 +67,7 @@ import org.apache.hadoop.hive.ql.optimiz
import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger;
import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
import org.apache.hadoop.hive.ql.optimizer.spark.SetSparkReducerParallelism;
-import org.apache.hadoop.hive.ql.optimizer.spark.SparkMapJoinOptimizer;
+import org.apache.hadoop.hive.ql.optimizer.spark.SparkJoinOptimizer;
import org.apache.hadoop.hive.ql.optimizer.spark.SparkReduceSinkMapJoinProc;
import org.apache.hadoop.hive.ql.optimizer.spark.SparkSkewJoinResolver;
import org.apache.hadoop.hive.ql.optimizer.spark.SparkSortMergeJoinFactory;
@@ -109,31 +110,29 @@ public class SparkCompiler extends TaskC
@Override
protected void optimizeOperatorPlan(ParseContext pCtx, Set<ReadEntity>
inputs,
Set<WriteEntity> outputs) throws SemanticException {
- // TODO: need to add spark specific optimization.
// Sequence of TableScan operators to be walked
Deque<Operator<? extends OperatorDesc>> deque = new LinkedList<Operator<?
extends OperatorDesc>>();
deque.addAll(pCtx.getTopOps().values());
- // Create the context for the walker
- OptimizeSparkProcContext procCtx
- = new OptimizeSparkProcContext(conf, pCtx, inputs, outputs, deque);
+ OptimizeSparkProcContext procCtx = new OptimizeSparkProcContext(conf,
pCtx, inputs, outputs, deque);
// create a walker which walks the tree in a DFS manner while maintaining
// the operator stack.
Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule,
NodeProcessor>();
opRules.put(new RuleRegExp("Set parallelism - ReduceSink",
- ReduceSinkOperator.getOperatorName() + "%"),
- new SetSparkReducerParallelism());
+ ReduceSinkOperator.getOperatorName() + "%"),
+ new SetSparkReducerParallelism());
- // TODO: need to research and verify support convert join to map join
optimization.
opRules.put(new RuleRegExp(new String("Convert Join to Map-join"),
- JoinOperator.getOperatorName() + "%"), new SparkMapJoinOptimizer());
+ JoinOperator.getOperatorName() + "%"), new SparkJoinOptimizer(pCtx));
// The dispatcher fires the processor corresponding to the closest matching
// rule and passes the context along
Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
- List<Node> topNodes = new ArrayList<Node>();
+ GraphWalker ogw = new DefaultGraphWalker(disp);
+
+ // Create a list of topop nodes
+ ArrayList<Node> topNodes = new ArrayList<Node>();
topNodes.addAll(pCtx.getTopOps().values());
- GraphWalker ogw = new ForwardWalker(disp);
ogw.startWalking(topNodes, null);
}
@@ -159,8 +158,7 @@ public class SparkCompiler extends TaskC
opRules.put(new RuleRegExp("Split Work - ReduceSink",
ReduceSinkOperator.getOperatorName() + "%"), genSparkWork);
- opRules.put(new RuleRegExp("No more walking on ReduceSink-MapJoin",
- MapJoinOperator.getOperatorName() + "%"), new
SparkReduceSinkMapJoinProc());
+ opRules.put(new TypeRule(MapJoinOperator.class), new
SparkReduceSinkMapJoinProc());
opRules.put(new RuleRegExp("Split Work + Move/Merge - FileSink",
FileSinkOperator.getOperatorName() + "%"),
@@ -185,6 +183,32 @@ public class SparkCompiler extends TaskC
}
);
+ /**
+ * SMB join case: (Big) (Small) (Small)
+ * TS TS TS
+ * \ | /
+ * \ DS DS
+ * \ | /
+ * SMBJoinOP
+ *
+ * Some of the other processors are expecting only one traversal beyond
SMBJoinOp.
+ * We need to traverse from the big-table path only, and stop traversing
on the small-table path once we reach SMBJoinOp.
+ */
+ opRules.put(new TypeRule(SMBMapJoinOperator.class),
+ new NodeProcessor() {
+ @Override
+ public Object process(Node currNode, Stack<Node> stack,
+ NodeProcessorCtx procCtx, Object... os) throws
SemanticException {
+ for (Node stackNode : stack) {
+ if (stackNode instanceof DummyStoreOperator) {
+ return true;
+ }
+ }
+ return false;
+ }
+ }
+ );
+
// The dispatcher fires the processor corresponding to the closest matching
// rule and passes the context along
Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx);
@@ -194,7 +218,6 @@ public class SparkCompiler extends TaskC
ogw.startWalking(topNodes, null);
-
// ------------------- Second Pass -----------------------
// SMB Join optimizations to add the "localWork" and bucketing data
structures to MapWork.
opRules.clear();
Modified:
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join32.q.out
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join32.q.out?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join32.q.out
(original)
+++
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join32.q.out
Thu Dec 18 19:58:10 2014
@@ -193,15 +193,18 @@ STAGE PLANS:
0 name (type: string)
1 name (type: string)
outputColumnNames: _col0, _col8
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
Group By Operator
aggregations: count(DISTINCT _col8)
keys: _col0 (type: string), _col8 (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: string), _col1 (type:
string)
sort order: ++
Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats:
NONE Column stats: NONE
Reducer 2
Reduce Operator Tree:
Group By Operator
@@ -209,11 +212,14 @@ STAGE PLANS:
keys: KEY._col0 (type: string)
mode: mergepartial
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column
stats: NONE
Select Operator
expressions: _col0 (type: string), _col1 (type: bigint)
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -312,15 +318,18 @@ STAGE PLANS:
0 name (type: string)
1 name (type: string)
outputColumnNames: _col0, _col8
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
Group By Operator
aggregations: count(DISTINCT _col8)
keys: _col0 (type: string), _col8 (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: string), _col1 (type:
string)
sort order: ++
Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats:
NONE Column stats: NONE
Reducer 2
Reduce Operator Tree:
Group By Operator
@@ -328,11 +337,14 @@ STAGE PLANS:
keys: KEY._col0 (type: string)
mode: mergepartial
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column
stats: NONE
Select Operator
expressions: _col0 (type: string), _col1 (type: bigint)
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -445,11 +457,14 @@ STAGE PLANS:
keys: KEY._col0 (type: string)
mode: mergepartial
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column
stats: NONE
Select Operator
expressions: _col0 (type: string), _col1 (type: bigint)
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Modified:
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join_stats.q.out
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join_stats.q.out?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join_stats.q.out
(original)
+++
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_join_stats.q.out
Thu Dec 18 19:58:10 2014
@@ -42,28 +42,28 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 3
+ Map 4
Map Operator Tree:
TableScan
- alias: src2
- Statistics: Num rows: 500 Data size: 5312 Basic stats:
COMPLETE Column stats: NONE
+ alias: smalltable
+ Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE
Column stats: NONE
Filter Operator
- predicate: key is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
+ predicate: UDFToDouble(key) is not null (type: boolean)
+ Statistics: Num rows: 3 Data size: 12 Basic stats:
COMPLETE Column stats: NONE
Spark HashTable Sink Operator
condition expressions:
- 0 {key}
- 1
+ 0 {_col0} {_col5}
+ 1 {key}
keys:
- 0 key (type: string)
- 1 key (type: string)
+ 0 (_col0 + _col5) (type: double)
+ 1 UDFToDouble(key) (type: double)
Local Work:
Map Reduce Local Work
Stage: Stage-1
Spark
Edges:
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL
SORT, 3)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL
SORT, 3)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -74,65 +74,63 @@ STAGE PLANS:
Filter Operator
predicate: key is not null (type: boolean)
Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- condition expressions:
- 0 {key}
- 1 {key}
- keys:
- 0 key (type: string)
- 1 key (type: string)
- outputColumnNames: _col0, _col5
- input vertices:
- 1 Map 3
- Statistics: Num rows: 275 Data size: 2921 Basic stats:
COMPLETE Column stats: NONE
- Filter Operator
- predicate: (_col0 + _col5) is not null (type: boolean)
- Statistics: Num rows: 138 Data size: 1465 Basic stats:
COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: (_col0 + _col5) (type: double)
- sort order: +
- Map-reduce partition columns: (_col0 + _col5) (type:
double)
- Statistics: Num rows: 138 Data size: 1465 Basic
stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: string), _col5
(type: string)
- Local Work:
- Map Reduce Local Work
- Map 4
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
+ Map 3
Map Operator Tree:
TableScan
- alias: smalltable
- Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE
Column stats: NONE
+ alias: src2
+ Statistics: Num rows: 500 Data size: 5312 Basic stats:
COMPLETE Column stats: NONE
Filter Operator
- predicate: UDFToDouble(key) is not null (type: boolean)
- Statistics: Num rows: 3 Data size: 12 Basic stats:
COMPLETE Column stats: NONE
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: UDFToDouble(key) (type: double)
+ key expressions: key (type: string)
sort order: +
- Map-reduce partition columns: UDFToDouble(key) (type:
double)
- Statistics: Num rows: 3 Data size: 12 Basic stats:
COMPLETE Column stats: NONE
- value expressions: key (type: string)
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
Reducer 2
+ Local Work:
+ Map Reduce Local Work
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
condition expressions:
- 0 {VALUE._col0} {VALUE._col5}
- 1 {VALUE._col0}
- outputColumnNames: _col0, _col5, _col10
- Statistics: Num rows: 151 Data size: 1611 Basic stats:
COMPLETE Column stats: NONE
- Select Operator
- expressions: _col0 (type: string), _col5 (type: string),
_col10 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 151 Data size: 1611 Basic stats:
COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
+ 0 {KEY.reducesinkkey0}
+ 1 {KEY.reducesinkkey0}
+ outputColumnNames: _col0, _col5
+ Statistics: Num rows: 275 Data size: 2921 Basic stats:
COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (_col0 + _col5) is not null (type: boolean)
+ Statistics: Num rows: 138 Data size: 1465 Basic stats:
COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0} {_col5}
+ 1 {key}
+ keys:
+ 0 (_col0 + _col5) (type: double)
+ 1 UDFToDouble(key) (type: double)
+ outputColumnNames: _col0, _col5, _col10
+ input vertices:
+ 1 Map 4
Statistics: Num rows: 151 Data size: 1611 Basic stats:
COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde:
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ Select Operator
+ expressions: _col0 (type: string), _col5 (type: string),
_col10 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 151 Data size: 1611 Basic stats:
COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 151 Data size: 1611 Basic stats:
COMPLETE Column stats: NONE
+ table:
+ input format:
org.apache.hadoop.mapred.TextInputFormat
+ output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde:
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
@@ -182,8 +180,7 @@ POSTHOOK: query: explain select src1.key
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
Stage-2 is a root stage
- Stage-3 depends on stages: Stage-2
- Stage-1 depends on stages: Stage-3
+ Stage-1 depends on stages: Stage-2
Stage-0 depends on stages: Stage-1
STAGE PLANS:
@@ -191,50 +188,45 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 5
+ Map 4
Map Operator Tree:
TableScan
- alias: smalltable2
- Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL
Column stats: NONE
+ alias: smalltable
+ Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE
Column stats: NONE
Filter Operator
predicate: UDFToDouble(key) is not null (type: boolean)
- Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
+ Statistics: Num rows: 3 Data size: 12 Basic stats:
COMPLETE Column stats: NONE
Spark HashTable Sink Operator
condition expressions:
- 0 {_col0} {_col5} {_col10}
- 1
+ 0 {_col0} {_col5}
+ 1 {key}
keys:
0 (_col0 + _col5) (type: double)
1 UDFToDouble(key) (type: double)
Local Work:
Map Reduce Local Work
-
- Stage: Stage-3
- Spark
-#### A masked pattern was here ####
- Vertices:
- Map 3
+ Map 5
Map Operator Tree:
TableScan
- alias: src2
- Statistics: Num rows: 500 Data size: 5312 Basic stats:
COMPLETE Column stats: NONE
+ alias: smalltable2
+ Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL
Column stats: NONE
Filter Operator
- predicate: key is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
+ predicate: UDFToDouble(key) is not null (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE
Column stats: NONE
Spark HashTable Sink Operator
condition expressions:
- 0 {key}
+ 0 {_col0} {_col5} {_col10}
1
keys:
- 0 key (type: string)
- 1 key (type: string)
+ 0 (_col0 + _col5) (type: double)
+ 1 UDFToDouble(key) (type: double)
Local Work:
Map Reduce Local Work
Stage: Stage-1
Spark
Edges:
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 4 (PARTITION-LEVEL
SORT, 3)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 3), Map 3 (PARTITION-LEVEL
SORT, 3)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -245,44 +237,24 @@ STAGE PLANS:
Filter Operator
predicate: key is not null (type: boolean)
Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- condition expressions:
- 0 {key}
- 1 {key}
- keys:
- 0 key (type: string)
- 1 key (type: string)
- outputColumnNames: _col0, _col5
- input vertices:
- 1 Map 3
- Statistics: Num rows: 275 Data size: 2921 Basic stats:
COMPLETE Column stats: NONE
- Filter Operator
- predicate: (_col0 + _col5) is not null (type: boolean)
- Statistics: Num rows: 138 Data size: 1465 Basic stats:
COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: (_col0 + _col5) (type: double)
- sort order: +
- Map-reduce partition columns: (_col0 + _col5) (type:
double)
- Statistics: Num rows: 138 Data size: 1465 Basic
stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: string), _col5
(type: string)
- Local Work:
- Map Reduce Local Work
- Map 4
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
+ Map 3
Map Operator Tree:
TableScan
- alias: smalltable
- Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE
Column stats: NONE
+ alias: src2
+ Statistics: Num rows: 500 Data size: 5312 Basic stats:
COMPLETE Column stats: NONE
Filter Operator
- predicate: UDFToDouble(key) is not null (type: boolean)
- Statistics: Num rows: 3 Data size: 12 Basic stats:
COMPLETE Column stats: NONE
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: UDFToDouble(key) (type: double)
+ key expressions: key (type: string)
sort order: +
- Map-reduce partition columns: UDFToDouble(key) (type:
double)
- Statistics: Num rows: 3 Data size: 12 Basic stats:
COMPLETE Column stats: NONE
- value expressions: key (type: string)
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats:
COMPLETE Column stats: NONE
Reducer 2
Local Work:
Map Reduce Local Work
@@ -291,37 +263,53 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
condition expressions:
- 0 {VALUE._col0} {VALUE._col5}
- 1 {VALUE._col0}
- outputColumnNames: _col0, _col5, _col10
- Statistics: Num rows: 151 Data size: 1611 Basic stats:
COMPLETE Column stats: NONE
+ 0 {KEY.reducesinkkey0}
+ 1 {KEY.reducesinkkey0}
+ outputColumnNames: _col0, _col5
+ Statistics: Num rows: 275 Data size: 2921 Basic stats:
COMPLETE Column stats: NONE
Filter Operator
predicate: (_col0 + _col5) is not null (type: boolean)
- Statistics: Num rows: 76 Data size: 810 Basic stats:
COMPLETE Column stats: NONE
+ Statistics: Num rows: 138 Data size: 1465 Basic stats:
COMPLETE Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
condition expressions:
- 0 {_col0} {_col5} {_col10}
- 1
+ 0 {_col0} {_col5}
+ 1 {key}
keys:
0 (_col0 + _col5) (type: double)
1 UDFToDouble(key) (type: double)
outputColumnNames: _col0, _col5, _col10
input vertices:
- 1 Map 5
- Statistics: Num rows: 83 Data size: 891 Basic stats:
COMPLETE Column stats: NONE
- Select Operator
- expressions: _col0 (type: string), _col5 (type: string),
_col10 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 83 Data size: 891 Basic stats:
COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
+ 1 Map 4
+ Statistics: Num rows: 151 Data size: 1611 Basic stats:
COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (_col0 + _col5) is not null (type: boolean)
+ Statistics: Num rows: 76 Data size: 810 Basic stats:
COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0} {_col5} {_col10}
+ 1
+ keys:
+ 0 (_col0 + _col5) (type: double)
+ 1 UDFToDouble(key) (type: double)
+ outputColumnNames: _col0, _col5, _col10
+ input vertices:
+ 1 Map 5
Statistics: Num rows: 83 Data size: 891 Basic stats:
COMPLETE Column stats: NONE
- table:
- input format:
org.apache.hadoop.mapred.TextInputFormat
- output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde:
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ Select Operator
+ expressions: _col0 (type: string), _col5 (type:
string), _col10 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 83 Data size: 891 Basic stats:
COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 83 Data size: 891 Basic
stats: COMPLETE Column stats: NONE
+ table:
+ input format:
org.apache.hadoop.mapred.TextInputFormat
+ output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde:
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
Modified:
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out
(original)
+++
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out
Thu Dec 18 19:58:10 2014
@@ -82,12 +82,15 @@ STAGE PLANS:
keys:
0 key (type: int)
1 key (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -95,11 +98,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -178,15 +184,18 @@ STAGE PLANS:
0 key (type: int)
1 key (type: int)
outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
keys: _col0 (type: int)
mode: hash
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -195,13 +204,17 @@ STAGE PLANS:
keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE
Column stats: NONE
Select Operator
+ Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE
Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 16 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 16 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 3
Reduce Operator Tree:
@@ -209,11 +222,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 16 Basic stats:
COMPLETE Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -319,15 +335,18 @@ STAGE PLANS:
0 key (type: int)
1 key (type: int)
outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
keys: _col0 (type: int)
mode: hash
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint)
Map 4
Map Operator Tree:
@@ -347,15 +366,18 @@ STAGE PLANS:
0 key (type: int)
1 key (type: int)
outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
keys: _col0 (type: int)
mode: hash
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -364,10 +386,12 @@ STAGE PLANS:
keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE
Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE
Column stats: NONE
value expressions: _col1 (type: bigint)
Reducer 3
Reduce Operator Tree:
@@ -378,11 +402,14 @@ STAGE PLANS:
0 {KEY.reducesinkkey0} {VALUE._col0}
1 {VALUE._col0}
outputColumnNames: _col0, _col1, _col3
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: bigint), _col3
(type: bigint)
outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 5 Data size: 41 Basic stats:
COMPLETE Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -394,10 +421,12 @@ STAGE PLANS:
keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE
Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE
Column stats: NONE
value expressions: _col1 (type: bigint)
Stage: Stage-0
@@ -500,12 +529,15 @@ STAGE PLANS:
keys:
0 _col0 (type: int)
1 _col0 (type: int)
+ Statistics: Num rows: 3 Data size: 23 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -513,11 +545,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -608,12 +643,15 @@ STAGE PLANS:
keys:
0 _col0 (type: int)
1 key (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -621,11 +659,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -740,12 +781,15 @@ STAGE PLANS:
keys:
0 _col0 (type: int)
1 _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 7 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -753,11 +797,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -862,12 +909,15 @@ STAGE PLANS:
keys:
0 _col0 (type: int)
1 _col0 (type: int)
+ Statistics: Num rows: 3 Data size: 23 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -875,11 +925,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1082,12 +1135,15 @@ STAGE PLANS:
keys:
0 _col0 (type: int)
1 key (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -1095,11 +1151,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1185,12 +1244,15 @@ STAGE PLANS:
0 _col0 (type: int)
1 _col0 (type: int)
2 _col0 (type: int)
+ Statistics: Num rows: 6 Data size: 46 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -1198,11 +1260,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1301,12 +1366,15 @@ STAGE PLANS:
keys:
0 _col0 (type: int)
1 key (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col0 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -1314,11 +1382,14 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1425,14 +1496,18 @@ STAGE PLANS:
0 key (type: int)
1 key (type: int)
outputColumnNames: _col0, _col1, _col6
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: string),
_col6 (type: string)
outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: string)
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
table:
input format:
org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1440,6 +1515,7 @@ STAGE PLANS:
name: default.dest1
File Output Operator
compressed: false
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
table:
input format:
org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1623,8 +1699,10 @@ STAGE PLANS:
0 key (type: int)
1 key (type: int)
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
table:
input format:
org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -1633,15 +1711,18 @@ STAGE PLANS:
Select Operator
expressions: _col0 (type: int)
outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
keys: _col0 (type: int)
mode: hash
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 38 Basic stats:
COMPLETE Column stats: NONE
value expressions: _col1 (type: bigint)
Reducer 2
Reduce Operator Tree:
@@ -1650,11 +1731,14 @@ STAGE PLANS:
keys: KEY._col0 (type: int)
mode: mergepartial
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: int), UDFToInteger(_col1) (type:
int)
outputColumnNames: _col0, _col1
+ Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
+ Statistics: Num rows: 2 Data size: 15 Basic stats:
COMPLETE Column stats: NONE
table:
input format: org.apache.hadoop.mapred.TextInputFormat
output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Modified:
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out
(original)
+++
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out
Thu Dec 18 19:58:10 2014
@@ -172,12 +172,15 @@ STAGE PLANS:
0 key (type: string)
1 key (type: string)
Position of Big Table: 1
+ Statistics: Num rows: 63 Data size: 6393 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
tag: -1
value expressions: _col0 (type: bigint)
auto parallelism: false
@@ -290,14 +293,17 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
#### A masked pattern was here ####
NumFilesPerFileSink: 1
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -404,12 +410,15 @@ STAGE PLANS:
0 key (type: string)
1 key (type: string)
Position of Big Table: 0
+ Statistics: Num rows: 63 Data size: 6393 Basic stats:
COMPLETE Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
tag: -1
value expressions: _col0 (type: bigint)
auto parallelism: false
@@ -522,14 +531,17 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
#### A masked pattern was here ####
NumFilesPerFileSink: 1
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -606,10 +618,94 @@ TOK_QUERY
STAGE DEPENDENCIES:
- Stage-1 is a root stage
+ 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 3
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE
Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 114 Basic stats:
COMPLETE Column stats: NONE
+ Spark HashTable Sink Operator
+ condition expressions:
+ 0
+ 1
+ keys:
+ 0 key (type: string)
+ 1 key (type: string)
+ Position of Big Table: 0
+ Local Work:
+ Map Reduce Local Work
+ Bucket Mapjoin Context:
+ Alias Bucket File Name Mapping:
+#### A masked pattern was here ####
+ Alias Bucket Output File Name Mapping:
+#### A masked pattern was here ####
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: ds=2008-04-08
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ partition values:
+ ds 2008-04-08
+ properties:
+ COLUMN_STATS_ACCURATE true
+ bucket_count 2
+ bucket_field_name key
+ columns key,value
+ columns.comments
+ columns.types string:string
+#### A masked pattern was here ####
+ name default.bucket_small
+ numFiles 2
+ numRows 0
+ partition_columns ds
+ partition_columns.types string
+ rawDataSize 0
+ serialization.ddl struct bucket_small { string key, string
value}
+ serialization.format 1
+ serialization.lib
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 114
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format:
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ SORTBUCKETCOLSPREFIX TRUE
+ bucket_count 2
+ bucket_field_name key
+ columns key,value
+ columns.comments
+ columns.types string:string
+#### A masked pattern was here ####
+ name default.bucket_small
+ partition_columns ds
+ partition_columns.types string
+ serialization.ddl struct bucket_small { string key,
string value}
+ serialization.format 1
+ serialization.lib
org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.bucket_small
+ name: default.bucket_small
+ Truncated Path -> Alias:
+ /bucket_small/ds=2008-04-08 [b]
+
Stage: Stage-1
Spark
Edges:
@@ -626,7 +722,7 @@ STAGE PLANS:
isSamplingPred: false
predicate: key is not null (type: boolean)
Statistics: Num rows: 58 Data size: 5812 Basic stats:
COMPLETE Column stats: NONE
- Sorted Merge Bucket Map Join Operator
+ Map Join Operator
condition map:
Inner Join 0 to 1
condition expressions:
@@ -635,16 +731,29 @@ STAGE PLANS:
keys:
0 key (type: string)
1 key (type: string)
+ input vertices:
+ 1 Map 3
Position of Big Table: 0
+ Statistics: Num rows: 63 Data size: 6393 Basic stats:
COMPLETE Column stats: NONE
+ BucketMapJoin: true
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
Reduce Output Operator
sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats:
COMPLETE Column stats: NONE
tag: -1
value expressions: _col0 (type: bigint)
auto parallelism: false
+ Local Work:
+ Map Reduce Local Work
+ Bucket Mapjoin Context:
+ Alias Bucket File Name Mapping:
+#### A masked pattern was here ####
+ Alias Bucket Output File Name Mapping:
+#### A masked pattern was here ####
Path -> Alias:
#### A masked pattern was here ####
Path -> Partition:
@@ -754,14 +863,17 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
Select Operator
expressions: _col0 (type: bigint)
outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
#### A masked pattern was here ####
NumFilesPerFileSink: 1
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE
Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.TextInputFormat
Modified:
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out
URL:
http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out?rev=1646523&r1=1646522&r2=1646523&view=diff
==============================================================================
---
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out
(original)
+++
hive/branches/spark/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out
Thu Dec 18 19:58:10 2014
@@ -138,7 +138,7 @@ POSTHOOK: query: load data local inpath
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@bucket_medium@ds=2008-04-08
-Warning: Map Join MAPJOIN[28][bigTable=?] in task 'Stage-1:MAPRED' is a cross
product
+Warning: Map Join MAPJOIN[26][bigTable=?] in task 'Stage-1:MAPRED' is a cross
product
PREHOOK: query: explain extended select count(*) FROM bucket_small a JOIN
bucket_medium b ON a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN
bucket_medium d ON c.key = b.key
PREHOOK: type: QUERY
POSTHOOK: query: explain extended select count(*) FROM bucket_small a JOIN
bucket_medium b ON a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN
bucket_medium d ON c.key = b.key
@@ -646,7 +646,7 @@ STAGE PLANS:
Processor Tree:
ListSink
-Warning: Map Join MAPJOIN[28][bigTable=?] in task 'Stage-1:MAPRED' is a cross
product
+Warning: Map Join MAPJOIN[26][bigTable=?] in task 'Stage-1:MAPRED' is a cross
product
PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_medium b ON
a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN bucket_medium d ON c.key
= b.key
PREHOOK: type: QUERY
PREHOOK: Input: default@bucket_big