Author: gunther
Date: Thu Mar 27 21:45:22 2014
New Revision: 1582488
URL: http://svn.apache.org/r1582488
Log:
HIVE-6753: Unions on Tez NPE when there's a mapjoin the union work (Gunther
Hagleitner, reviewed by Vikram Dixit K)
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
hive/trunk/ql/src/test/queries/clientpositive/tez_union.q
hive/trunk/ql/src/test/results/clientpositive/tez/load_dyn_part1.q.out
hive/trunk/ql/src/test/results/clientpositive/tez/load_dyn_part3.q.out
hive/trunk/ql/src/test/results/clientpositive/tez/mrr.q.out
hive/trunk/ql/src/test/results/clientpositive/tez/tez_dml.q.out
hive/trunk/ql/src/test/results/clientpositive/tez/tez_union.q.out
hive/trunk/ql/src/test/results/clientpositive/tez/vectorization_15.q.out
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
URL:
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java?rev=1582488&r1=1582487&r2=1582488&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
(original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java
Thu Mar 27 21:45:22 2014
@@ -123,6 +123,9 @@ public class TezProcessor implements Log
@Override
public void run(Map<String, LogicalInput> inputs, Map<String, LogicalOutput>
outputs)
throws Exception {
+
+ Exception processingException = null;
+
try{
perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.TEZ_RUN_PROCESSOR);
// in case of broadcast-join read the broadcast edge inputs
@@ -160,9 +163,20 @@ public class TezProcessor implements Log
//done - output does not need to be committed as hive does not use
outputcommitter
perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_RUN_PROCESSOR);
+ } catch (Exception e) {
+ processingException = e;
} finally {
- if(rproc != null){
- rproc.close();
+ try {
+ if(rproc != null){
+ rproc.close();
+ }
+ } catch (Exception e) {
+ if (processingException == null) {
+ processingException = e;
+ }
+ }
+ if (processingException != null) {
+ throw processingException;
}
}
}
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
URL:
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java?rev=1582488&r1=1582487&r2=1582488&view=diff
==============================================================================
---
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
(original)
+++
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
Thu Mar 27 21:45:22 2014
@@ -42,6 +42,7 @@ import org.apache.hadoop.hive.ql.plan.Ex
import org.apache.hadoop.hive.ql.plan.HashTableDummyDesc;
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.TableDesc;
import org.apache.hadoop.hive.ql.plan.TezWork;
import org.apache.hadoop.hive.ql.plan.TezWork.EdgeType;
@@ -63,12 +64,16 @@ public class ReduceSinkMapJoinProc imple
public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx
procContext, Object... nodeOutputs)
throws SemanticException {
GenTezProcContext context = (GenTezProcContext) procContext;
+ MapJoinOperator mapJoinOp = (MapJoinOperator)nd;
+
+ if (stack.size() < 2 || !(stack.get(stack.size() - 2) instanceof
ReduceSinkOperator)) {
+ context.currentMapJoinOperators.add(mapJoinOp);
+ return null;
+ }
+
context.preceedingWork = null;
context.currentRootOperator = null;
- MapJoinOperator mapJoinOp = (MapJoinOperator)nd;
- Operator<? extends OperatorDesc> childOp =
mapJoinOp.getChildOperators().get(0);
-
ReduceSinkOperator parentRS = (ReduceSinkOperator)stack.get(stack.size() -
2);
// remember the original parent list before we start modifying it.
@@ -77,66 +82,72 @@ public class ReduceSinkMapJoinProc imple
context.mapJoinParentMap.put(mapJoinOp, parents);
}
- BaseWork myWork = null;
+ List<BaseWork> mapJoinWork = null;
- while (childOp != null) {
- if ((childOp instanceof ReduceSinkOperator) || (childOp instanceof
FileSinkOperator)) {
- /*
- * if there was a pre-existing work generated for the big-table
mapjoin side,
- * we need to hook the work generated for the RS (associated with the
RS-MJ pattern)
- * with the pre-existing work.
- *
- * Otherwise, we need to associate that the reduce sink/file sink
down the MJ path
- * to be linked to the RS work (associated with the RS-MJ pattern).
- *
- */
-
- myWork = context.operatorWorkMap.get(childOp);
- BaseWork parentWork = context.operatorWorkMap.get(parentRS);
-
- // set the link between mapjoin and parent vertex
- int pos = context.mapJoinParentMap.get(mapJoinOp).indexOf(parentRS);
- if (pos == -1) {
- throw new SemanticException("Cannot find position of parent in
mapjoin");
- }
- LOG.debug("Mapjoin "+mapJoinOp+", pos: "+pos+" -->
"+parentWork.getName());
- mapJoinOp.getConf().getParentToInput().put(pos, parentWork.getName());
+ /*
+ * if there was a pre-existing work generated for the big-table mapjoin
side,
+ * we need to hook the work generated for the RS (associated with the
RS-MJ pattern)
+ * with the pre-existing work.
+ *
+ * Otherwise, we need to associate that the mapjoin op
+ * to be linked to the RS work (associated with the RS-MJ pattern).
+ *
+ */
+ mapJoinWork = context.mapJoinWorkMap.get(mapJoinOp);
+ BaseWork parentWork;
+ if (context.unionWorkMap.containsKey(parentRS)) {
+ parentWork = context.unionWorkMap.get(parentRS);
+ } else {
+ assert context.childToWorkMap.get(parentRS).size() == 1;
+ parentWork = context.childToWorkMap.get(parentRS).get(0);
+ }
- if (myWork != null) {
- // link the work with the work associated with the reduce sink that
triggered this rule
- TezWork tezWork = context.currentTask.getWork();
- tezWork.connect(parentWork, myWork, EdgeType.BROADCAST_EDGE);
-
- // remember the output name of the reduce sink
- parentRS.getConf().setOutputName(myWork.getName());
- context.connectedReduceSinks.add(parentRS);
+ // set the link between mapjoin and parent vertex
+ int pos = context.mapJoinParentMap.get(mapJoinOp).indexOf(parentRS);
+ if (pos == -1) {
+ throw new SemanticException("Cannot find position of parent in mapjoin");
+ }
+ LOG.debug("Mapjoin "+mapJoinOp+", pos: "+pos+" --> "+parentWork.getName());
+ mapJoinOp.getConf().getParentToInput().put(pos, parentWork.getName());
+ if (mapJoinWork != null) {
+ for (BaseWork myWork: mapJoinWork) {
+ // link the work with the work associated with the reduce sink that
triggered this rule
+ TezWork tezWork = context.currentTask.getWork();
+ LOG.debug("connecting "+parentWork.getName()+" with
"+myWork.getName());
+ tezWork.connect(parentWork, myWork, EdgeType.BROADCAST_EDGE);
+
+ ReduceSinkOperator r = null;
+ 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 {
- List<BaseWork> linkWorkList = context.linkOpWithWorkMap.get(childOp);
- if (linkWorkList == null) {
- linkWorkList = new ArrayList<BaseWork>();
- }
- linkWorkList.add(parentWork);
- context.linkOpWithWorkMap.put(childOp, linkWorkList);
-
- List<ReduceSinkOperator> reduceSinks
- = context.linkWorkWithReduceSinkMap.get(parentWork);
- if (reduceSinks == null) {
- reduceSinks = new ArrayList<ReduceSinkOperator>();
- }
- reduceSinks.add(parentRS);
- context.linkWorkWithReduceSinkMap.put(parentWork, reduceSinks);
+ r = parentRS;
}
-
- break;
+ // remember the output name of the reduce sink
+ r.getConf().setOutputName(myWork.getName());
+ context.connectedReduceSinks.add(r);
}
+ }
- if ((childOp.getChildOperators() != null) &&
(childOp.getChildOperators().size() >= 1)) {
- childOp = childOp.getChildOperators().get(0);
- } else {
- break;
- }
+ // remember in case we need to connect additional work later
+ List<BaseWork> linkWorkList = context.linkOpWithWorkMap.get(mapJoinOp);
+ if (linkWorkList == null) {
+ linkWorkList = new ArrayList<BaseWork>();
}
+ linkWorkList.add(parentWork);
+ context.linkOpWithWorkMap.put(mapJoinOp, linkWorkList);
+
+ List<ReduceSinkOperator> reduceSinks
+ = context.linkWorkWithReduceSinkMap.get(parentWork);
+ if (reduceSinks == null) {
+ reduceSinks = new ArrayList<ReduceSinkOperator>();
+ }
+ reduceSinks.add(parentRS);
+ context.linkWorkWithReduceSinkMap.put(parentWork, reduceSinks);
// create the dummy operators
List<Operator<? extends OperatorDesc>> dummyOperators =
@@ -178,17 +189,18 @@ public class ReduceSinkMapJoinProc imple
// the "work" needs to know about the dummy operators. They have to be
separately initialized
// at task startup
- if (myWork != null) {
- myWork.addDummyOp(dummyOp);
- } else {
- List<Operator<?>> dummyList = dummyOperators;
- if (context.linkChildOpWithDummyOp.containsKey(childOp)) {
- dummyList = context.linkChildOpWithDummyOp.get(childOp);
+ if (mapJoinWork != null) {
+ for (BaseWork myWork: mapJoinWork) {
+ myWork.addDummyOp(dummyOp);
}
- dummyList.add(dummyOp);
- context.linkChildOpWithDummyOp.put(childOp, dummyList);
}
+ if (context.linkChildOpWithDummyOp.containsKey(mapJoinOp)) {
+ for (Operator<?> op: context.linkChildOpWithDummyOp.get(mapJoinOp)) {
+ dummyOperators.add(op);
+ }
+ }
+ context.linkChildOpWithDummyOp.put(mapJoinOp, dummyOperators);
+
return true;
}
-
}
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
URL:
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java?rev=1582488&r1=1582487&r2=1582488&view=diff
==============================================================================
---
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
(original)
+++
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
Thu Mar 27 21:45:22 2014
@@ -95,12 +95,15 @@ public class GenTezProcContext implement
// map join work
public final Map<BaseWork, List<ReduceSinkOperator>>
linkWorkWithReduceSinkMap;
- // a map that maintains operator (file-sink or reduce-sink) to work mapping
- public final Map<Operator<?>, BaseWork> operatorWorkMap;
+ // map that says which mapjoin belongs to which work item
+ public final Map<MapJoinOperator, List<BaseWork>> mapJoinWorkMap;
// a map to keep track of which root generated which work
public final Map<Operator<?>, BaseWork> rootToWorkMap;
+ // a map to keep track of which child generated with work
+ public final Map<Operator<?>, List<BaseWork>> childToWorkMap;
+
// we need to keep the original list of operators in the map join to know
// what position in the mapjoin the different parent work items will have.
public final Map<MapJoinOperator, List<Operator<?>>> mapJoinParentMap;
@@ -111,10 +114,14 @@ public class GenTezProcContext implement
// used to group dependent tasks for multi table inserts
public final DependencyCollectionTask dependencyTask;
+ // remember map joins as we encounter them.
+ public final Set<MapJoinOperator> currentMapJoinOperators;
+
// used to hook up unions
public final Map<Operator<?>, BaseWork> unionWorkMap;
public final List<UnionOperator> currentUnionOperators;
public final Set<BaseWork> workWithUnionOperators;
+ public final Set<ReduceSinkOperator> clonedReduceSinks;
// we link filesink that will write to the same final location
public final Map<Path, List<FileSinkDesc>> linkedFileSinks;
@@ -139,15 +146,18 @@ public class GenTezProcContext implement
this.leafOperatorToFollowingWork = new HashMap<Operator<?>, BaseWork>();
this.linkOpWithWorkMap = new HashMap<Operator<?>, List<BaseWork>>();
this.linkWorkWithReduceSinkMap = new HashMap<BaseWork,
List<ReduceSinkOperator>>();
- this.operatorWorkMap = new HashMap<Operator<?>, BaseWork>();
+ this.mapJoinWorkMap = new HashMap<MapJoinOperator, List<BaseWork>>();
this.rootToWorkMap = new HashMap<Operator<?>, BaseWork>();
+ this.childToWorkMap = new HashMap<Operator<?>, List<BaseWork>>();
this.mapJoinParentMap = new HashMap<MapJoinOperator, List<Operator<?>>>();
+ this.currentMapJoinOperators = new HashSet<MapJoinOperator>();
this.linkChildOpWithDummyOp = new HashMap<Operator<?>,
List<Operator<?>>>();
this.dependencyTask = (DependencyCollectionTask)
TaskFactory.get(new DependencyCollectionWork(), conf);
this.unionWorkMap = new HashMap<Operator<?>, BaseWork>();
this.currentUnionOperators = new LinkedList<UnionOperator>();
this.workWithUnionOperators = new HashSet<BaseWork>();
+ this.clonedReduceSinks = new HashSet<ReduceSinkOperator>();
this.linkedFileSinks = new HashMap<Path, List<FileSinkDesc>>();
this.fileSinkSet = new HashSet<FileSinkOperator>();
this.connectedReduceSinks = new HashSet<ReduceSinkOperator>();
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
URL:
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java?rev=1582488&r1=1582487&r2=1582488&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
(original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java Thu
Mar 27 21:45:22 2014
@@ -29,7 +29,9 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
+import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
import org.apache.hadoop.hive.ql.exec.TableScanOperator;
import org.apache.hadoop.hive.ql.exec.UnionOperator;
@@ -41,6 +43,7 @@ import org.apache.hadoop.hive.ql.optimiz
import org.apache.hadoop.hive.ql.plan.BaseWork;
import org.apache.hadoop.hive.ql.plan.MapWork;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
import org.apache.hadoop.hive.ql.plan.ReduceWork;
import org.apache.hadoop.hive.ql.plan.TezWork;
import org.apache.hadoop.hive.ql.plan.UnionWork;
@@ -87,6 +90,12 @@ public class GenTezWork implements NodeP
LOG.debug("Root operator: " + root);
LOG.debug("Leaf operator: " + operator);
+ if (context.clonedReduceSinks.contains(operator)) {
+ // if we're visiting a terminal we've created ourselves,
+ // just skip and keep going
+ return null;
+ }
+
TezWork tezWork = context.currentTask.getWork();
// Right now the work graph is pretty simple. If there is no
@@ -112,42 +121,75 @@ public class GenTezWork implements NodeP
}
context.rootToWorkMap.put(root, work);
}
- context.operatorWorkMap.put(operator, work);
- /*
- * this happens in case of map join operations.
- * The tree looks like this:
- *
- * RS <--- we are here perhaps
- * |
- * MapJoin
- * / \
- * RS TS
- * /
- * TS
- *
- * If we are at the RS pointed above, and we may have already visited the
- * RS following the TS, we have already generated work for the TS-RS.
- * We need to hook the current work to this generated work.
- */
- List<BaseWork> linkWorkList = context.linkOpWithWorkMap.get(operator);
- if (linkWorkList != null) {
- if (context.linkChildOpWithDummyOp.containsKey(operator)) {
- for (Operator<?> dummy: context.linkChildOpWithDummyOp.get(operator)) {
- work.addDummyOp((HashTableDummyOperator) dummy);
+ if (!context.childToWorkMap.containsKey(operator)) {
+ List<BaseWork> workItems = new LinkedList<BaseWork>();
+ workItems.add(work);
+ context.childToWorkMap.put(operator, workItems);
+ } else {
+ context.childToWorkMap.get(operator).add(work);
+ }
+
+ // remember which mapjoin operator links with which work
+ if (!context.currentMapJoinOperators.isEmpty()) {
+ for (MapJoinOperator mj: context.currentMapJoinOperators) {
+ LOG.debug("Processing map join: " + mj);
+ // remember the mapping in case we scan another branch of the
+ // mapjoin later
+ if (!context.mapJoinWorkMap.containsKey(mj)) {
+ List<BaseWork> workItems = new LinkedList<BaseWork>();
+ workItems.add(work);
+ context.mapJoinWorkMap.put(mj, workItems);
+ } else {
+ context.mapJoinWorkMap.get(mj).add(work);
}
- }
- for (BaseWork parentWork : linkWorkList) {
- tezWork.connect(parentWork, work, EdgeType.BROADCAST_EDGE);
- // need to set up output name for reduce sink now that we know the name
- // of the downstream work
- for (ReduceSinkOperator r:
- context.linkWorkWithReduceSinkMap.get(parentWork)) {
- r.getConf().setOutputName(work.getName());
- context.connectedReduceSinks.add(r);
+ /*
+ * this happens in case of map join operations.
+ * The tree looks like this:
+ *
+ * RS <--- we are here perhaps
+ * |
+ * MapJoin
+ * / \
+ * RS TS
+ * /
+ * TS
+ *
+ * If we are at the RS pointed above, and we may have already visited
the
+ * RS following the TS, we have already generated work for the TS-RS.
+ * We need to hook the current work to this generated work.
+ */
+ List<BaseWork> linkWorkList = context.linkOpWithWorkMap.get(mj);
+ if (linkWorkList != null) {
+ if (context.linkChildOpWithDummyOp.containsKey(mj)) {
+ for (Operator<?> dummy: context.linkChildOpWithDummyOp.get(mj)) {
+ work.addDummyOp((HashTableDummyOperator) dummy);
+ }
+ }
+ for (BaseWork parentWork : linkWorkList) {
+ LOG.debug("connecting "+parentWork.getName()+" with
"+work.getName());
+ tezWork.connect(parentWork, work, EdgeType.BROADCAST_EDGE);
+
+ // need to set up output name for reduce sink now that we know the
name
+ // of the downstream work
+ for (ReduceSinkOperator r:
+ context.linkWorkWithReduceSinkMap.get(parentWork)) {
+ if (r.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)r.getConf().clone(),
r.getParentOperators());
+ context.clonedReduceSinks.add(r);
+ }
+ r.getConf().setOutputName(work.getName());
+ context.connectedReduceSinks.add(r);
+ }
+ }
}
}
+ // clear out the set. we don't need it anymore.
+ context.currentMapJoinOperators.clear();
}
// This is where we cut the tree as described above. We also remember that
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
URL:
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java?rev=1582488&r1=1582487&r2=1582488&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
(original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java Thu
Mar 27 21:45:22 2014
@@ -143,7 +143,6 @@ public class TezCompiler extends TaskCom
genTezWork);
opRules.put(new RuleRegExp("No more walking on ReduceSink-MapJoin",
- ReduceSinkOperator.getOperatorName() + "%" +
MapJoinOperator.getOperatorName() + "%"), new ReduceSinkMapJoinProc());
opRules.put(new RuleRegExp("Split Work + Move/Merge - FileSink",
@@ -154,7 +153,7 @@ public class TezCompiler extends TaskCom
TableScanOperator.getOperatorName() + "%"),
new ProcessAnalyzeTable(GenTezUtils.getUtils()));
- opRules.put(new RuleRegExp("Handle union",
+ opRules.put(new RuleRegExp("Remember union",
UnionOperator.getOperatorName() + "%"), new NodeProcessor()
{
@Override
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
URL:
http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java?rev=1582488&r1=1582487&r2=1582488&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
(original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
Thu Mar 27 21:45:22 2014
@@ -132,6 +132,7 @@ public class ReduceSinkDesc extends Abst
desc.setValueSerializeInfo((TableDesc) getValueSerializeInfo().clone());
desc.setNumBuckets(numBuckets);
desc.setBucketCols(bucketCols);
+ desc.setStatistics(this.getStatistics());
return desc;
}
Modified: hive/trunk/ql/src/test/queries/clientpositive/tez_union.q
URL:
http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/tez_union.q?rev=1582488&r1=1582487&r2=1582488&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/tez_union.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/tez_union.q Thu Mar 27
21:45:22 2014
@@ -5,10 +5,14 @@ select s1.key as key, s1.value as value
UNION ALL
select s2.key as key, s2.value as value from src s2;
+create table ut as
select s1.key as key, s1.value as value from src s1 join src s3 on
s1.key=s3.key
UNION ALL
select s2.key as key, s2.value as value from src s2;
+select * from ut order by key, value limit 20;
+drop table ut;
+
set hive.auto.convert.join=false;
explain
@@ -16,6 +20,75 @@ with u as (select * from src union all s
select count(*) from (select u1.key as k1, u2.key as k2 from
u as u1 join u as u2 on (u1.key = u2.key)) a;
+create table ut as
with u as (select * from src union all select * from src)
-select count(*) from (select u1.key as k1, u2.key as k2 from
+select count(*) as cnt from (select u1.key as k1, u2.key as k2 from
u as u1 join u as u2 on (u1.key = u2.key)) a;
+
+select * from ut order by cnt limit 20;
+drop table ut;
+
+set hive.auto.convert.join=true;
+
+explain select s1.key as skey, u1.key as ukey from
+src s1
+join (select * from src union all select * from src) u1 on s1.key = u1.key;
+
+create table ut as
+select s1.key as skey, u1.key as ukey from
+src s1
+join (select * from src union all select * from src) u1 on s1.key = u1.key;
+
+select * from ut order by skey, ukey limit 20;
+drop table ut;
+
+explain select s1.key as skey, u1.key as ukey, s8.key as lkey from
+src s1
+join (select s2.key as key from src s2 join src s3 on s2.key = s3.key
+ union all select s4.key from src s4 join src s5 on s4.key = s5.key
+ union all select s6.key from src s6 join src s7 on s6.key = s7.key) u1
on (s1.key = u1.key)
+join src s8 on (u1.key = s8.key)
+order by lkey;
+
+create table ut as
+select s1.key as skey, u1.key as ukey, s8.key as lkey from
+src s1
+join (select s2.key as key from src s2 join src s3 on s2.key = s3.key
+ union all select s4.key from src s4 join src s5 on s4.key = s5.key
+ union all select s6.key from src s6 join src s7 on s6.key = s7.key) u1
on (s1.key = u1.key)
+join src s8 on (u1.key = s8.key)
+order by lkey;
+
+select * from ut order by skey, ukey, lkey limit 100;
+
+drop table ut;
+
+explain
+select s2.key as key from src s2 join src s3 on s2.key = s3.key
+union all select s4.key from src s4 join src s5 on s4.key = s5.key;
+
+create table ut as
+select s2.key as key from src s2 join src s3 on s2.key = s3.key
+union all select s4.key from src s4 join src s5 on s4.key = s5.key;
+
+select * from ut order by key limit 30;
+
+drop table ut;
+
+explain
+select * from
+(select * from src union all select * from src) u
+left outer join src s on u.key = s.key;
+
+explain
+select u.key as ukey, s.key as skey from
+(select * from src union all select * from src) u
+right outer join src s on u.key = s.key;
+
+create table ut as
+select u.key as ukey, s.key as skey from
+(select * from src union all select * from src) u
+right outer join src s on u.key = s.key;
+
+select * from ut order by ukey, skey limit 20;
+drop table ut;
\ No newline at end of file