getReducerInfo(TezWork tzWrk, String vertex, String prntVertex)
+ throws SemanticException {
+ BaseWork prntWork = tzWrk.getWorkMap().get(prntVertex);
+ return new ExtractReduceSinkInfo(vertex).analyze(prntWork);
+ }
+
+ /*
+ * Given a Work descriptor and the TaskName for the work
+ * this is responsible to check each MapJoinOp for cross products.
+ * The analyze call returns the warnings list.
+ *
+ * For MR the taskname is the StageName, for Tez it is the vertex name.
+ */
+ class MapJoinCheck implements NodeProcessor, NodeProcessorCtx {
+
+ final List warnings;
+ final String taskName;
+
+ MapJoinCheck(String taskName) {
+ this.taskName = taskName;
+ warnings = new ArrayList();
+ }
+
+ List analyze(BaseWork work) throws SemanticException {
+ Map opRules = new LinkedHashMap();
+ opRules.put(new RuleRegExp("R1", MapJoinOperator.getOperatorName()
+ + "%"), this);
+ Dispatcher disp = new DefaultRuleDispatcher(new NoopProcessor(), opRules, this);
+ GraphWalker ogw = new DefaultGraphWalker(disp);
+ ArrayList topNodes = new ArrayList();
+ topNodes.addAll(work.getAllRootOperators());
+ ogw.startWalking(topNodes, null);
+ return warnings;
+ }
+
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+ @SuppressWarnings("unchecked")
+ AbstractMapJoinOperator extends MapJoinDesc> mjOp = (AbstractMapJoinOperator extends MapJoinDesc>) nd;
+ MapJoinDesc mjDesc = mjOp.getConf();
+
+ String bigTablAlias = mjDesc.getBigTableAlias();
+ if ( bigTablAlias == null ) {
+ Operator extends OperatorDesc> parent = null;
+ for(Operator extends OperatorDesc> op : mjOp.getParentOperators() ) {
+ if ( op instanceof TableScanOperator ) {
+ parent = op;
+ }
+ }
+ if ( parent != null) {
+ TableScanDesc tDesc = ((TableScanOperator)parent).getConf();
+ bigTablAlias = tDesc.getAlias();
+ }
+ }
+ bigTablAlias = bigTablAlias == null ? "?" : bigTablAlias;
+
+ List joinExprs = mjDesc.getKeys().values().iterator().next();
+
+ if ( joinExprs.size() == 0 ) {
+ warnings.add(
+ String.format("Map Join %s[bigTable=%s] in task '%s' is a cross product",
+ mjOp.toString(), bigTablAlias, taskName));
+ }
+
+ return null;
+ }
+ }
+
+ /*
+ * for a given Work Descriptor, it extracts information about the ReduceSinkOps
+ * in the Work. For Tez, you can restrict it to ReduceSinks for a particular output
+ * vertex.
+ */
+ static class ExtractReduceSinkInfo implements NodeProcessor, NodeProcessorCtx {
+
+ static class Info {
+ List keyCols;
+ List inputAliases;
+
+ Info(List keyCols, List inputAliases) {
+ this.keyCols = keyCols;
+ this.inputAliases = inputAliases == null ? new ArrayList() : inputAliases;
+ }
+
+ Info(List keyCols, String[] inputAliases) {
+ this.keyCols = keyCols;
+ this.inputAliases = inputAliases == null ? new ArrayList() : Arrays.asList(inputAliases);
+ }
+ }
+
+ final String outputTaskName;
+ final Map reduceSinkInfo;
+
+ ExtractReduceSinkInfo(String parentTaskName) {
+ this.outputTaskName = parentTaskName;
+ reduceSinkInfo = new HashMap();
+ }
+
+ Map analyze(BaseWork work) throws SemanticException {
+ Map opRules = new LinkedHashMap();
+ opRules.put(new RuleRegExp("R1", ReduceSinkOperator.getOperatorName()
+ + "%"), this);
+ Dispatcher disp = new DefaultRuleDispatcher(new NoopProcessor(), opRules, this);
+ GraphWalker ogw = new DefaultGraphWalker(disp);
+ ArrayList topNodes = new ArrayList();
+ topNodes.addAll(work.getAllRootOperators());
+ ogw.startWalking(topNodes, null);
+ return reduceSinkInfo;
+ }
+
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+ ReduceSinkOperator rsOp = (ReduceSinkOperator) nd;
+ ReduceSinkDesc rsDesc = rsOp.getConf();
+
+ if ( outputTaskName != null ) {
+ String rOutputName = rsDesc.getOutputName();
+ if ( rOutputName == null || !outputTaskName.equals(rOutputName)) {
+ return null;
+ }
+ }
+
+ reduceSinkInfo.put(rsDesc.getTag(),
+ new Info(rsDesc.getKeyCols(), rsOp.getInputAliases()));
+
+ return null;
+ }
+ }
+
+ static class NoopProcessor implements NodeProcessor {
+ @Override
+ public final Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+ return nd;
+ }
+ }
+}
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java
index d593d08..cf049b2 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/PhysicalOptimizer.java
@@ -78,6 +78,10 @@ private void initialize(HiveConf hiveConf) {
resolvers.add(new BucketingSortingInferenceOptimizer());
}
+ if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_CHECK_CROSS_PRODUCT)) {
+ resolvers.add(new CrossProductCheck());
+ }
+
// Vectorization should be the last optimization, because it doesn't modify the plan
// or any operators. It makes a very low level transformation to the expressions to
// run in the vectorized mode.
diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
index e2ea0d4..c09c5be 100644
--- ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
+++ ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
@@ -59,6 +59,7 @@
import org.apache.hadoop.hive.ql.optimizer.ConvertJoinMapJoin;
import org.apache.hadoop.hive.ql.optimizer.ReduceSinkMapJoinProc;
import org.apache.hadoop.hive.ql.optimizer.SetReducerParallelism;
+import org.apache.hadoop.hive.ql.optimizer.physical.CrossProductCheck;
import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext;
import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer;
import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger;
@@ -245,6 +246,11 @@ protected void optimizeTaskPlan(List> rootTasks, Pa
Context ctx) throws SemanticException {
PhysicalContext physicalCtx = new PhysicalContext(conf, pCtx, pCtx.getContext(), rootTasks,
pCtx.getFetchTask());
+
+ if (conf.getBoolVar(HiveConf.ConfVars.HIVE_CHECK_CROSS_PRODUCT)) {
+ physicalCtx = new CrossProductCheck().resolve(physicalCtx);
+ }
+
if (conf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED)) {
(new Vectorizer()).resolve(physicalCtx);
}
diff --git ql/src/test/queries/clientpositive/cross_product_check_1.q ql/src/test/queries/clientpositive/cross_product_check_1.q
new file mode 100644
index 0000000..fb38c94
--- /dev/null
+++ ql/src/test/queries/clientpositive/cross_product_check_1.q
@@ -0,0 +1,26 @@
+
+create table A as
+select * from src;
+
+create table B as
+select * from src
+limit 10;
+
+set hive.auto.convert.join.noconditionaltask.size=100;
+
+explain select * from A join B;
+
+explain select * from B d1 join B d2 on d1.key = d2.key join A;
+
+explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1;
+
+explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1;
+
+explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1;
+
+
diff --git ql/src/test/queries/clientpositive/cross_product_check_2.q ql/src/test/queries/clientpositive/cross_product_check_2.q
new file mode 100644
index 0000000..479d571
--- /dev/null
+++ ql/src/test/queries/clientpositive/cross_product_check_2.q
@@ -0,0 +1,27 @@
+create table A as
+select * from src;
+
+create table B as
+select * from src
+limit 10;
+
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask=true;
+set hive.auto.convert.join.noconditionaltask.size=10000000;
+
+explain select * from A join B;
+
+explain select * from B d1 join B d2 on d1.key = d2.key join A;
+
+explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1;
+
+explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1;
+
+explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1;
+
+
diff --git ql/src/test/results/clientpositive/auto_join0.q.out ql/src/test/results/clientpositive/auto_join0.q.out
index 4d1cc4d..b3c6d12 100644
--- ql/src/test/results/clientpositive/auto_join0.q.out
+++ ql/src/test/results/clientpositive/auto_join0.q.out
@@ -1,3 +1,4 @@
+Warning: Map Join MAPJOIN[24][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain
select sum(hash(a.k1,a.v1,a.k2, a.v2))
from (
@@ -137,6 +138,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Map Join MAPJOIN[24][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: select sum(hash(a.k1,a.v1,a.k2, a.v2))
from (
SELECT src1.key as k1, src1.value as v1,
diff --git ql/src/test/results/clientpositive/auto_join23.q.out ql/src/test/results/clientpositive/auto_join23.q.out
index 86d51e4..80ee1fb 100644
--- ql/src/test/results/clientpositive/auto_join23.q.out
+++ ql/src/test/results/clientpositive/auto_join23.q.out
@@ -1,3 +1,4 @@
+Warning: Map Join MAPJOIN[14][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain
SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
PREHOOK: type: QUERY
@@ -78,6 +79,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Map Join MAPJOIN[14][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
PREHOOK: type: QUERY
PREHOOK: Input: default@src
diff --git ql/src/test/results/clientpositive/auto_join_filters.q.out ql/src/test/results/clientpositive/auto_join_filters.q.out
index 7c13c46..690fa1e 100644
--- ql/src/test/results/clientpositive/auto_join_filters.q.out
+++ ql/src/test/results/clientpositive/auto_join_filters.q.out
@@ -13,6 +13,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in3.txt' INTO TABLE my
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@myinput1
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -22,6 +23,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
3078400
+Warning: Map Join MAPJOIN[17][bigTable=a] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -31,6 +33,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
4937935
+Warning: Map Join MAPJOIN[17][bigTable=b] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -40,6 +43,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
3080335
+Warning: Shuffle Join JOIN[4][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a FULL OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -293,6 +297,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table sm
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@smb_input2
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -302,6 +307,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
3078400
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -311,6 +317,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
3078400
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -320,6 +327,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
3078400
+Warning: Shuffle Join JOIN[10][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a FULL OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
diff --git ql/src/test/results/clientpositive/auto_join_nulls.q.out ql/src/test/results/clientpositive/auto_join_nulls.q.out
index f183075..b61f8a0 100644
--- ql/src/test/results/clientpositive/auto_join_nulls.q.out
+++ ql/src/test/results/clientpositive/auto_join_nulls.q.out
@@ -13,6 +13,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in1.txt' INTO TABLE my
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@myinput1
+Warning: Map Join MAPJOIN[13][bigTable=b] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -22,6 +23,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
13630578
+Warning: Map Join MAPJOIN[13][bigTable=a] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -31,6 +33,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
13630578
+Warning: Map Join MAPJOIN[13][bigTable=b] in task 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a RIGHT OUTER JOIN myinput1 b
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
index 79f40ac..e468d04 100644
--- ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
+++ ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
@@ -135,6 +135,7 @@ POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3out
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@bucket_medium@ds=2008-04-08
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-2: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
@@ -641,6 +642,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-2: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
diff --git ql/src/test/results/clientpositive/create_view.q.out ql/src/test/results/clientpositive/create_view.q.out
index 68eeeaf..246a7e1 100644
--- ql/src/test/results/clientpositive/create_view.q.out
+++ ql/src/test/results/clientpositive/create_view.q.out
@@ -519,6 +519,7 @@ POSTHOOK: Input: default@table1
POSTHOOK: Input: default@view4
POSTHOOK: Output: default@view5
POSTHOOK: Lineage: table1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+Warning: Shuffle Join JOIN[6][tables = [v1, v2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM view5
PREHOOK: type: QUERY
PREHOOK: Input: default@table1
diff --git ql/src/test/results/clientpositive/cross_join.q.out ql/src/test/results/clientpositive/cross_join.q.out
index 294b8d5..8309fda 100644
--- ql/src/test/results/clientpositive/cross_join.q.out
+++ ql/src/test/results/clientpositive/cross_join.q.out
@@ -1,3 +1,4 @@
+Warning: Shuffle Join JOIN[4][tables = [src, src2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: -- current
explain select src.key from src join src src2
PREHOOK: type: QUERY
@@ -50,6 +51,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[4][tables = [src, src2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: -- ansi cross join
explain select src.key from src cross join src src2
PREHOOK: type: QUERY
diff --git ql/src/test/results/clientpositive/cross_product_check_1.q.out ql/src/test/results/clientpositive/cross_product_check_1.q.out
new file mode 100644
index 0000000..eb69348
--- /dev/null
+++ ql/src/test/results/clientpositive/cross_product_check_1.q.out
@@ -0,0 +1,581 @@
+PREHOOK: query: create table A as
+select * from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+POSTHOOK: query: create table A as
+select * from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@A
+PREHOOK: query: create table B as
+select * from src
+limit 10
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+POSTHOOK: query: create table B as
+select * from src
+limit 10
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@B
+Warning: Shuffle Join JOIN[4][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join B
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join B
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 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
+ limit: -1
+
+Warning: Shuffle Join JOIN[8][tables = [d1, d2, a]] in Stage 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ TableScan
+ alias: d2
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string)
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1} {VALUE._col4} {VALUE._col5}
+ 1 {VALUE._col0} {VALUE._col1}
+ outputColumnNames: _col0, _col1, _col4, _col5, _col8, _col9
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string), _col8 (type: string), _col9 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 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
+ limit: -1
+
+Warning: Shuffle Join JOIN[14][tables = [a, od1]] in Stage 'Stage-3:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-3 depends on stages: Stage-2
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string)
+ TableScan
+ alias: d2
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-3
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0}
+ outputColumnNames: _col0, _col1, _col4
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 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
+ limit: -1
+
+Warning: Shuffle Join JOIN[14][tables = [a, od1]] in Stage 'Stage-3:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[4][tables = [d1, d2]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-3 depends on stages: Stage-2
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string)
+ TableScan
+ alias: d2
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-3
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0}
+ outputColumnNames: _col0, _col1, _col4
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 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
+ limit: -1
+
+Warning: Shuffle Join JOIN[19][tables = [ss, od1]] in Stage 'Stage-3:MAPRED' is a cross product
+PREHOOK: query: explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-3 depends on stages: Stage-2, Stage-5
+ Stage-5 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string)
+ TableScan
+ alias: d2
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-3
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1 {VALUE._col0}
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 275 Data size: 2921 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-5
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string)
+ outputColumnNames: key
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: key (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
diff --git ql/src/test/results/clientpositive/cross_product_check_2.q.out ql/src/test/results/clientpositive/cross_product_check_2.q.out
new file mode 100644
index 0000000..960ad1f
--- /dev/null
+++ ql/src/test/results/clientpositive/cross_product_check_2.q.out
@@ -0,0 +1,610 @@
+PREHOOK: query: create table A as
+select * from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+POSTHOOK: query: create table A as
+select * from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@A
+PREHOOK: query: create table B as
+select * from src
+limit 10
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+POSTHOOK: query: create table B as
+select * from src
+limit 10
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@B
+Warning: Map Join MAPJOIN[7][bigTable=a] in task 'Stage-3:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join B
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join B
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-3 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-3
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {key} {value}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 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
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ b
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ b
+ TableScan
+ alias: b
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+Warning: Map Join MAPJOIN[13][bigTable=?] in task 'Stage-5:MAPRED' is a cross product
+PREHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-5 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-5
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {key} {value}
+ keys:
+ 0 key (type: string)
+ 1 key (type: string)
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0} {_col1} {_col4} {_col5}
+ 1 {key} {value}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col4, _col5, _col8, _col9
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string), _col8 (type: string), _col9 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 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
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ a
+ Fetch Operator
+ limit: -1
+ d1
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ a
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ d1
+ TableScan
+ alias: d1
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+Warning: Map Join MAPJOIN[22][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-2 is a root stage
+ Stage-6 depends on stages: Stage-2
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1
+ keys:
+ 0 key (type: string)
+ 1 key (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ od1:d1
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ od1:d1
+ TableScan
+ alias: d1
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-6
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {_col0}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col4
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 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
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ a
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ a
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+Warning: Map Join MAPJOIN[22][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[23][bigTable=d2] in task 'Stage-2:MAPRED' is a cross product
+PREHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-2 is a root stage
+ Stage-6 depends on stages: Stage-2
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ od1:d1
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ od1:d1
+ TableScan
+ alias: d1
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-6
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {_col0}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col4
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 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
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ a
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ a
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+Warning: Map Join MAPJOIN[58][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[88][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[19][tables = [ss, od1]] in Stage 'Stage-3:MAPRED' is a cross product
+PREHOOK: query: explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-5 is a root stage
+ Stage-8 depends on stages: Stage-2, Stage-5 , consists of Stage-6, Stage-7, Stage-3
+ Stage-6 has a backup stage: Stage-3
+ Stage-7 has a backup stage: Stage-3
+ Stage-3
+ Stage-2 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-5
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string)
+ outputColumnNames: key
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: key (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-8
+ Conditional Operator
+
+ Stage: Stage-6
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0}
+ 1 {_col0}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ 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
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ $INTNAME
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ $INTNAME
+ TableScan
+
+ Stage: Stage-7
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0}
+ 1 {_col0}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ 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
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ $INTNAME1
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ $INTNAME1
+ TableScan
+
+ Stage: Stage-3
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1 {VALUE._col0}
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 275 Data size: 2921 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-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1
+ keys:
+ 0 key (type: string)
+ 1 key (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 5 Data size: 51 Basic stats: COMPLETE Column stats: NONE
+ Local Work:
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ od1:d1
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ od1:d1
+ TableScan
+ alias: d1
+ Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
diff --git ql/src/test/results/clientpositive/input23.q.out ql/src/test/results/clientpositive/input23.q.out
index fefed0f..a5864f5 100644
--- ql/src/test/results/clientpositive/input23.q.out
+++ ql/src/test/results/clientpositive/input23.q.out
@@ -1,3 +1,4 @@
+Warning: Shuffle Join JOIN[4][tables = [a]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: explain extended
select * from srcpart a join srcpart b where a.ds = '2008-04-08' and a.hr = '11' and b.ds = '2008-04-08' and b.hr = '14' limit 5
PREHOOK: type: QUERY
@@ -165,6 +166,7 @@ STAGE PLANS:
Fetch Operator
limit: 5
+Warning: Shuffle Join JOIN[4][tables = [a]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: select * from srcpart a join srcpart b where a.ds = '2008-04-08' and a.hr = '11' and b.ds = '2008-04-08' and b.hr = '14' limit 5
PREHOOK: type: QUERY
PREHOOK: Input: default@srcpart
diff --git ql/src/test/results/clientpositive/join0.q.out ql/src/test/results/clientpositive/join0.q.out
index 0a3792a..c21b1ce 100644
--- ql/src/test/results/clientpositive/join0.q.out
+++ ql/src/test/results/clientpositive/join0.q.out
@@ -1,3 +1,4 @@
+Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: EXPLAIN
SELECT src1.key as k1, src1.value as v1,
src2.key as k2, src2.value as v2 FROM
@@ -95,6 +96,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: EXPLAIN FORMATTED
SELECT src1.key as k1, src1.value as v1,
src2.key as k2, src2.value as v2 FROM
@@ -112,6 +114,7 @@ SELECT src1.key as k1, src1.value as v1,
SORT BY k1, v1, k2, v2
POSTHOOK: type: QUERY
{"STAGE PLANS":{"Stage-2":{"Map Reduce":{"Reduce Operator Tree:":{"Extract":{"children":{"File Output Operator":{"Statistics:":"Num rows: 9 Data size: 1983 Basic stats: COMPLETE Column stats: NONE","compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}}},"Statistics:":"Num rows: 9 Data size: 1983 Basic stats: COMPLETE Column stats: NONE"}},"Map Operator Tree:":[{"TableScan":{"children":{"Reduce Output Operator":{"sort order:":"++++","value expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","Statistics:":"Num rows: 9 Data size: 1983 Basic stats: COMPLETE Column stats: NONE","key expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)"}}}}]}},"Stage-1":{"Map Reduce":{"Reduce Operator Tree:":{"Join Operator":{"outputColumnNames:":["_col0","_col1","_col2","_col3"],"children":{"Select Operator":{"expressions:":"_col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)","outputColumnNames:":["_col0","_col1","_col2","_col3"],"children":{"File Output Operator":{"compressed:":"false","table:":{"serde:":"org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe","input format:":"org.apache.hadoop.mapred.SequenceFileInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"}}},"Statistics:":"Num rows: 9 Data size: 1983 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 9 Data size: 1983 Basic stats: COMPLETE Column stats: NONE","condition map:":[{"":"Inner Join 0 to 1"}],"condition expressions:":{"1":"{VALUE._col0} {VALUE._col1}","0":"{VALUE._col0} {VALUE._col1}"}}},"Map Operator Tree:":[{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 9 Data size: 1803 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 9 Data size: 1803 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 9 Data size: 1803 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE"}},{"TableScan":{"alias:":"src","children":{"Filter Operator":{"predicate:":"(key < 10) (type: boolean)","children":{"Select Operator":{"expressions:":"key (type: string), value (type: string)","outputColumnNames:":["_col0","_col1"],"children":{"Reduce Output Operator":{"sort order:":"","value expressions:":"_col0 (type: string), _col1 (type: string)","Statistics:":"Num rows: 9 Data size: 1803 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 9 Data size: 1803 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 9 Data size: 1803 Basic stats: COMPLETE Column stats: NONE"}},"Statistics:":"Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE"}}]}},"Stage-0":{"Fetch Operator":{"limit:":"-1"}}},"STAGE DEPENDENCIES":{"Stage-2":{"DEPENDENT STAGES":"Stage-1"},"Stage-1":{"ROOT STAGE":"TRUE"},"Stage-0":{"ROOT STAGE":"TRUE"}}}
+Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT src1.key as k1, src1.value as v1,
src2.key as k2, src2.value as v2 FROM
(SELECT * FROM src WHERE src.key < 10) src1
diff --git ql/src/test/results/clientpositive/join23.q.out ql/src/test/results/clientpositive/join23.q.out
index b2c7175..ccf2f1e 100644
--- ql/src/test/results/clientpositive/join23.q.out
+++ ql/src/test/results/clientpositive/join23.q.out
@@ -1,3 +1,4 @@
+Warning: Shuffle Join JOIN[4][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: EXPLAIN
SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
PREHOOK: type: QUERY
@@ -77,6 +78,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[4][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
PREHOOK: type: QUERY
PREHOOK: Input: default@src
diff --git ql/src/test/results/clientpositive/join_alt_syntax.q.out ql/src/test/results/clientpositive/join_alt_syntax.q.out
index b56f4f7..45df31e 100644
--- ql/src/test/results/clientpositive/join_alt_syntax.q.out
+++ ql/src/test/results/clientpositive/join_alt_syntax.q.out
@@ -39,6 +39,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwri
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@part
+Warning: Shuffle Join JOIN[4][tables = [p1, p2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: explain select p1.p_name, p2.p_name
from part p1 , part p2
PREHOOK: type: QUERY
@@ -242,6 +243,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select p1.p_name, p2.p_name, p3.p_name
from part p1 , part p2 , part p3
where p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name
diff --git ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out
index 9c75d42..107973d 100644
--- ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out
+++ ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out
@@ -175,6 +175,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part p2 join part p3 on p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name
PREHOOK: type: QUERY
@@ -267,6 +268,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part p2 join part p3 on p2.p_partkey = 1 and p3.p_name = p2.p_name
PREHOOK: type: QUERY
diff --git ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out
index 1d7a911..87e211f 100644
--- ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out
+++ ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out
@@ -185,6 +185,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part p2 join part p3
where p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name
@@ -282,6 +283,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part p2 join part p3
where p2.p_partkey = 1 and p3.p_name = p2.p_name
diff --git ql/src/test/results/clientpositive/join_cond_pushdown_unqual1.q.out ql/src/test/results/clientpositive/join_cond_pushdown_unqual1.q.out
index d66b3a2..2462d27 100644
--- ql/src/test/results/clientpositive/join_cond_pushdown_unqual1.q.out
+++ ql/src/test/results/clientpositive/join_cond_pushdown_unqual1.q.out
@@ -229,6 +229,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part2 p2 join part3 p3 on p2_partkey + p_partkey = p1.p_partkey and p3_name = p2_name
PREHOOK: type: QUERY
@@ -321,6 +322,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part2 p2 join part3 p3 on p2_partkey = 1 and p3_name = p2_name
PREHOOK: type: QUERY
diff --git ql/src/test/results/clientpositive/join_cond_pushdown_unqual3.q.out ql/src/test/results/clientpositive/join_cond_pushdown_unqual3.q.out
index 0522d2d..b66b14d 100644
--- ql/src/test/results/clientpositive/join_cond_pushdown_unqual3.q.out
+++ ql/src/test/results/clientpositive/join_cond_pushdown_unqual3.q.out
@@ -239,6 +239,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part2 p2 join part3 p3
where p2_partkey + p1.p_partkey = p1.p_partkey and p3_name = p2_name
@@ -336,6 +337,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[5][tables = [p1, p2]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part2 p2 join part3 p3
where p2_partkey = 1 and p3_name = p2_name
diff --git ql/src/test/results/clientpositive/join_filters.q.out ql/src/test/results/clientpositive/join_filters.q.out
index c5c0615..d219da1 100644
--- ql/src/test/results/clientpositive/join_filters.q.out
+++ ql/src/test/results/clientpositive/join_filters.q.out
@@ -13,6 +13,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in3.txt' INTO TABLE my
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@myinput1
+Warning: Shuffle Join JOIN[10][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -22,6 +23,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
100 100 100 100
+Warning: Shuffle Join JOIN[7][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -34,6 +36,7 @@ NULL 40 NULL NULL
12 35 NULL NULL
48 NULL NULL NULL
100 100 100 100
+Warning: Shuffle Join JOIN[7][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -46,6 +49,7 @@ NULL NULL NULL 40
NULL NULL 12 35
NULL NULL 48 NULL
100 100 100 100
+Warning: Shuffle Join JOIN[4][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a FULL OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -325,6 +329,7 @@ NULL NULL NULL NULL NULL 40
NULL NULL NULL NULL 12 35
NULL NULL NULL NULL 48 NULL
100 100 100 100 100 100
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -370,6 +375,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
100 100 100 100
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -705,6 +711,7 @@ NULL NULL 48 NULL
NULL NULL 148 NULL
100 100 100 100
200 200 200 200
+Warning: Shuffle Join JOIN[10][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -714,6 +721,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
100 100 100 100
+Warning: Shuffle Join JOIN[10][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -723,6 +731,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
100 100 100 100
+Warning: Shuffle Join JOIN[10][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -732,6 +741,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
100 100 100 100
+Warning: Shuffle Join JOIN[10][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a FULL OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -939,6 +949,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
100 100 100 100 100 100
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -984,6 +995,7 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@myinput1
#### A masked pattern was here ####
100 100 100 100
+Warning: Map Join MAPJOIN[17][bigTable=?] in task 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
diff --git ql/src/test/results/clientpositive/join_nulls.q.out ql/src/test/results/clientpositive/join_nulls.q.out
index 3d35e43..d9f5f0f 100644
--- ql/src/test/results/clientpositive/join_nulls.q.out
+++ ql/src/test/results/clientpositive/join_nulls.q.out
@@ -13,6 +13,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/in1.txt' INTO TABLE my
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@myinput1
+Warning: Shuffle Join JOIN[4][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -30,6 +31,7 @@ NULL 35 100 100
100 100 NULL 35
100 100 48 NULL
100 100 100 100
+Warning: Shuffle Join JOIN[4][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -47,6 +49,7 @@ NULL 35 100 100
100 100 NULL 35
100 100 48 NULL
100 100 100 100
+Warning: Shuffle Join JOIN[4][tables = [a, b]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
@@ -273,6 +276,7 @@ POSTHOOK: Input: default@myinput1
NULL NULL NULL NULL 48 NULL
NULL 35 NULL 35 NULL 35
100 100 100 100 100 100
+Warning: Map Join MAPJOIN[9][bigTable=b] in task 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC
PREHOOK: type: QUERY
PREHOOK: Input: default@myinput1
diff --git ql/src/test/results/clientpositive/no_hooks.q.out ql/src/test/results/clientpositive/no_hooks.q.out
index 795ee88..4b60e71 100644
--- ql/src/test/results/clientpositive/no_hooks.q.out
+++ ql/src/test/results/clientpositive/no_hooks.q.out
@@ -1,3 +1,4 @@
+Warning: Shuffle Join JOIN[4][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
POSTHOOK: query: EXPLAIN
SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
POSTHOOK: type: QUERY
@@ -74,6 +75,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[4][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
POSTHOOK: query: SELECT * FROM src src1 JOIN src src2 WHERE src1.key < 10 and src2.key < 10 SORT BY src1.key, src1.value, src2.key, src2.value
POSTHOOK: type: QUERY
POSTHOOK: Input: default@src
diff --git ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out
index 98900ad..a9c1629 100644
--- ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out
+++ ql/src/test/results/clientpositive/nonblock_op_deduplicate.q.out
@@ -35,6 +35,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[11][tables = [tmp2, tmp3]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: -- This test query is introduced for HIVE-4968.
-- First, we do not convert the join to MapJoin.
EXPLAIN
@@ -163,6 +164,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[11][tables = [tmp2, tmp3]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count
FROM (SELECT tmp2.key as key, tmp2.value as value, tmp3.count as count
FROM (SELECT *
@@ -210,6 +212,7 @@ POSTHOOK: Input: default@src1
406 val_406 25
66 val_66 25
98 val_98 25
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
PREHOOK: query: -- Then, we convert the join to MapJoin.
EXPLAIN
SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count
@@ -337,6 +340,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Map Join MAPJOIN[21][bigTable=?] in task 'Stage-3:MAPRED' is a cross product
PREHOOK: query: SELECT tmp4.key as key, tmp4.value as value, tmp4.count as count
FROM (SELECT tmp2.key as key, tmp2.value as value, tmp3.count as count
FROM (SELECT *
diff --git ql/src/test/results/clientpositive/ppd_repeated_alias.q.out ql/src/test/results/clientpositive/ppd_repeated_alias.q.out
index d63dfd2..e57db82 100644
--- ql/src/test/results/clientpositive/ppd_repeated_alias.q.out
+++ ql/src/test/results/clientpositive/ppd_repeated_alias.q.out
@@ -228,6 +228,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[9][tables = [c, d]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: -- Q4: here, the filter c.bar should be created under the first join but above the second
explain select c.foo, d.bar from (select c.foo, b.bar, c.blah from pokes c left outer join pokes b on c.foo=b.foo) c left outer join pokes d where d.foo=1 and c.bar=2
PREHOOK: type: QUERY
diff --git ql/src/test/results/clientpositive/print_header.q.out ql/src/test/results/clientpositive/print_header.q.out
index 4e939e8..408b957 100644
--- ql/src/test/results/clientpositive/print_header.q.out
+++ ql/src/test/results/clientpositive/print_header.q.out
@@ -1,3 +1,4 @@
+Warning: Shuffle Join JOIN[8][tables = [src1, src2]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT src1.key as k1, src1.value as v1,
src2.key as k2, src2.value as v2 FROM
(SELECT * FROM src WHERE src.key < 10) src1
diff --git ql/src/test/results/clientpositive/sample8.q.out ql/src/test/results/clientpositive/sample8.q.out
index 17a8dfe..0b6f246 100644
--- ql/src/test/results/clientpositive/sample8.q.out
+++ ql/src/test/results/clientpositive/sample8.q.out
@@ -436,6 +436,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[6][tables = [s, t]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: SELECT s.key, s.value
FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON key) s
JOIN srcpart TABLESAMPLE (BUCKET 1 OUT OF 10 ON key) t
diff --git ql/src/test/results/clientpositive/subquery_multiinsert.q.out ql/src/test/results/clientpositive/subquery_multiinsert.q.out
index 8269599..b7e44c9 100644
--- ql/src/test/results/clientpositive/subquery_multiinsert.q.out
+++ ql/src/test/results/clientpositive/subquery_multiinsert.q.out
@@ -26,6 +26,7 @@ POSTHOOK: type: CREATETABLE
POSTHOOK: Output: database:default
POSTHOOK: Output: default@src_5
RUN: Stage-0:DDL
+Warning: Shuffle Join JOIN[29][tables = [b, sq_2_notin_nullcheck]] in Stage 'Stage-4:MAPRED' is a cross product
PREHOOK: query: explain
from src b
INSERT OVERWRITE TABLE src_4
@@ -298,6 +299,7 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+Warning: Shuffle Join JOIN[29][tables = [b, sq_2_notin_nullcheck]] in Stage 'Stage-4:MAPRED' is a cross product
PREHOOK: query: from src b
INSERT OVERWRITE TABLE src_4
select *
@@ -497,6 +499,8 @@ POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:s
199 val_199
199 val_199
2 val_2
+Warning: Map Join MAPJOIN[103][bigTable=?] in task 'Stage-12:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[29][tables = [b, sq_2_notin_nullcheck]] in Stage 'Stage-4:MAPRED' is a cross product
PREHOOK: query: explain
from src b
INSERT OVERWRITE TABLE src_4
@@ -815,6 +819,8 @@ STAGE PLANS:
Stage: Stage-3
Stats-Aggr Operator
+Warning: Map Join MAPJOIN[103][bigTable=?] in task 'Stage-12:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[29][tables = [b, sq_2_notin_nullcheck]] in Stage 'Stage-4:MAPRED' is a cross product
PREHOOK: query: from src b
INSERT OVERWRITE TABLE src_4
select *
diff --git ql/src/test/results/clientpositive/subquery_notin.q.out ql/src/test/results/clientpositive/subquery_notin.q.out
index 4769eed..735aae0 100644
--- ql/src/test/results/clientpositive/subquery_notin.q.out
+++ ql/src/test/results/clientpositive/subquery_notin.q.out
@@ -92,6 +92,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRIT
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@lineitem
+Warning: Shuffle Join JOIN[18][tables = [src, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: -- non agg, non corr
explain
select *
@@ -246,6 +247,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[18][tables = [src, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: select *
from src
where src.key not in ( select key from src s1 where s1.key > '2')
@@ -379,6 +381,7 @@ POSTHOOK: Input: default@src
199 val_199
199 val_199
2 val_2
+Warning: Shuffle Join JOIN[26][tables = [b, sq_1_notin_nullcheck]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: -- non agg, corr
explain
select p_mfgr, b.p_name, p_size
@@ -581,6 +584,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[26][tables = [b, sq_1_notin_nullcheck]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: select p_mfgr, b.p_name, p_size
from part b
where b.p_name not in
@@ -619,6 +623,7 @@ Manufacturer#4 almond azure aquamarine papaya violet 12
Manufacturer#5 almond antique blue firebrick mint 31
Manufacturer#5 almond aquamarine dodger light gainsboro 46
Manufacturer#5 almond azure blanched chiffon midnight 23
+Warning: Shuffle Join JOIN[34][tables = [part, sq_1_notin_nullcheck]] in Stage 'Stage-6:MAPRED' is a cross product
PREHOOK: query: -- agg, non corr
explain
select p_name, p_size
@@ -864,6 +869,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[34][tables = [part, sq_1_notin_nullcheck]] in Stage 'Stage-7:MAPRED' is a cross product
PREHOOK: query: select p_name, p_size
from
part where part.p_size not in
@@ -910,6 +916,7 @@ almond aquamarine sandy cyan gainsboro 18
almond aquamarine yellow dodger mint 7
almond azure aquamarine papaya violet 12
almond azure blanched chiffon midnight 23
+Warning: Shuffle Join JOIN[34][tables = [b, sq_1_notin_nullcheck]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: -- agg, corr
explain
select p_mfgr, p_name, p_size
@@ -1183,6 +1190,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[34][tables = [b, sq_1_notin_nullcheck]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: select p_mfgr, p_name, p_size
from part b where b.p_size not in
(select min(p_size)
@@ -1223,6 +1231,7 @@ Manufacturer#5 almond antique medium spring khaki 6
Manufacturer#5 almond azure blanched chiffon midnight 23
Manufacturer#5 almond antique blue firebrick mint 31
Manufacturer#5 almond aquamarine dodger light gainsboro 46
+Warning: Shuffle Join JOIN[18][tables = [li, sq_1_notin_nullcheck]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: -- non agg, non corr, Group By in Parent Query
select li.l_partkey, count(*)
from lineitem li
@@ -1257,6 +1266,7 @@ POSTHOOK: Input: default@lineitem
139636 1
175839 1
182052 1
+Warning: Shuffle Join JOIN[18][tables = [src, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: -- alternate not in syntax
select *
from src
@@ -1414,6 +1424,7 @@ POSTHOOK: type: CREATEVIEW
POSTHOOK: Input: default@src
POSTHOOK: Input: default@t1_v
POSTHOOK: Output: default@T2_v
+Warning: Shuffle Join JOIN[24][tables = [t1_v, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain
select *
from T1_v where T1_v.key not in (select T2_v.key from T2_v)
@@ -1572,6 +1583,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[24][tables = [t1_v, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: select *
from T1_v where T1_v.key not in (select T2_v.key from T2_v)
PREHOOK: type: QUERY
diff --git ql/src/test/results/clientpositive/subquery_notin_having.q.out ql/src/test/results/clientpositive/subquery_notin_having.q.out
index a8b668b..0cc593c 100644
--- ql/src/test/results/clientpositive/subquery_notin_having.q.out
+++ ql/src/test/results/clientpositive/subquery_notin_having.q.out
@@ -39,6 +39,7 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwri
POSTHOOK: type: LOAD
#### A masked pattern was here ####
POSTHOOK: Output: default@part
+Warning: Shuffle Join JOIN[22][tables = [gby_sq1, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: -- non agg, non corr
explain
select key, count(*)
@@ -230,6 +231,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[32][tables = [gby_sq1, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: -- non agg, corr
explain
select b.p_mfgr, min(p_retailprice)
@@ -493,6 +495,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[32][tables = [gby_sq1, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: select b.p_mfgr, min(p_retailprice)
from part b
group by b.p_mfgr
@@ -517,6 +520,7 @@ POSTHOOK: Input: default@part
#### A masked pattern was here ####
Manufacturer#1 1173.15
Manufacturer#2 1690.68
+Warning: Shuffle Join JOIN[30][tables = [gby_sq1, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: -- agg, non corr
explain
select b.p_mfgr, min(p_retailprice)
@@ -785,6 +789,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[30][tables = [gby_sq1, sq_1_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: select b.p_mfgr, min(p_retailprice)
from part b
group by b.p_mfgr
diff --git ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
index d375159..87fbd3c 100644
--- ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
+++ ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
@@ -751,6 +751,7 @@ STAGE PLANS:
Fetch Operator
limit: -1
+Warning: Shuffle Join JOIN[26][tables = [b, sq_1_notin_nullcheck]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: -- non agg, corr
explain
select p_mfgr, b.p_name, p_size
diff --git ql/src/test/results/clientpositive/subquery_views.q.out ql/src/test/results/clientpositive/subquery_views.q.out
index f5bc3e7..f1fd6ee 100644
--- ql/src/test/results/clientpositive/subquery_views.q.out
+++ ql/src/test/results/clientpositive/subquery_views.q.out
@@ -59,6 +59,8 @@ where b.key not in
POSTHOOK: type: CREATEVIEW
POSTHOOK: Input: default@src
POSTHOOK: Output: default@cv2
+Warning: Shuffle Join JOIN[42][tables = [b, sq_1_notin_nullcheck]] in Stage 'Stage-5:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[18][tables = [b, sq_1_notin_nullcheck]] in Stage 'Stage-1:MAPRED' is a cross product
PREHOOK: query: select *
from cv2 where cv2.key in (select key from cv2 c where c.key < '11')
PREHOOK: type: QUERY
diff --git ql/src/test/results/clientpositive/tez/cross_product_check_1.q.out ql/src/test/results/clientpositive/tez/cross_product_check_1.q.out
new file mode 100644
index 0000000..841344c
--- /dev/null
+++ ql/src/test/results/clientpositive/tez/cross_product_check_1.q.out
@@ -0,0 +1,549 @@
+PREHOOK: query: create table A as
+select * from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+POSTHOOK: query: create table A as
+select * from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@A
+PREHOOK: query: create table B as
+select * from src
+limit 10
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+POSTHOOK: query: create table B as
+select * from src
+limit 10
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@B
+Warning: Shuffle Join JOIN[4][tables = [a, b]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: explain select * from A join B
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join B
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 6393 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
+ limit: -1
+
+Warning: Shuffle Join JOIN[8][tables = [null, a]] in Stage 'Reducer 3' is a cross product
+PREHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Statistics: Num rows: 0 Data size: 125 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 125 Basic stats: PARTIAL Column stats: NONE
+ value expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string)
+ Reducer 3
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1} {VALUE._col4} {VALUE._col5}
+ 1 {VALUE._col0} {VALUE._col1}
+ outputColumnNames: _col0, _col1, _col4, _col5, _col8, _col9
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string), _col8 (type: string), _col9 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 6393 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
+ limit: -1
+
+Warning: Shuffle Join JOIN[14][tables = [a, od1]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (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 3
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Reducer 4
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0}
+ outputColumnNames: _col0, _col1, _col4
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 6393 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
+ limit: -1
+
+Warning: Shuffle Join JOIN[4][tables = [d1, d2]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join JOIN[14][tables = [a, od1]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (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 3
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Reducer 4
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0}
+ outputColumnNames: _col0, _col1, _col4
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 6393 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
+ limit: -1
+
+Warning: Shuffle Join JOIN[19][tables = [ss, od1]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+ Reducer 7 <- Map 6 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ value expressions: key (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string)
+ outputColumnNames: key
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: key (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (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 3
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Reducer 4
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1 {VALUE._col0}
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 31 Data size: 3196 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 31 Data size: 3196 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 3196 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
+ Reducer 7
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
diff --git ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out
new file mode 100644
index 0000000..905133c
--- /dev/null
+++ ql/src/test/results/clientpositive/tez/cross_product_check_2.q.out
@@ -0,0 +1,515 @@
+PREHOOK: query: create table A as
+select * from src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+POSTHOOK: query: create table A as
+select * from src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@A
+PREHOOK: query: create table B as
+select * from src
+limit 10
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+POSTHOOK: query: create table B as
+select * from src
+limit 10
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@B
+Warning: Map Join MAPJOIN[7][bigTable=a] in task 'Map 2' is a cross product
+PREHOOK: query: explain select * from A join B
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join B
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 2 <- Map 1 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {key} {value}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 6393 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
+ limit: -1
+
+Warning: Map Join MAPJOIN[12][bigTable=a] in task 'Map 3' is a cross product
+PREHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from B d1 join B d2 on d1.key = d2.key join A
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+ Map 3 <- Map 1 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {key} {value}
+ keys:
+ 0 key (type: string)
+ 1 key (type: string)
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Statistics: Num rows: 0 Data size: 125 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 125 Basic stats: PARTIAL Column stats: NONE
+ value expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string)
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE
+ value expressions: key (type: string), value (type: string)
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0} {_col1} {_col4} {_col5}
+ 1 {key} {value}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col4, _col5, _col8, _col9
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string), _col5 (type: string), _col8 (type: string), _col9 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 6393 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
+ limit: -1
+
+Warning: Map Join MAPJOIN[19][bigTable=a] in task 'Map 4' is a cross product
+PREHOOK: query: explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join
+ (select d1.key
+ from B d1 join B d2 on d1.key = d2.key
+ where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Map 4 <- Reducer 2 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1
+ keys:
+ 0 key (type: string)
+ 1 key (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {_col0}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col4
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 6393 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
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ value expressions: _col0 (type: string)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+Warning: Map Join MAPJOIN[18][bigTable=d1] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[19][bigTable=a] in task 'Map 4' is a cross product
+PREHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Map 4 <- Reducer 2 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: COMPLETE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {_col0}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col4
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col4 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 31 Data size: 6393 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 6393 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
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ value expressions: _col0 (type: string)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+Warning: Map Join MAPJOIN[24][bigTable=?] in task 'Reducer 5' is a cross product
+PREHOOK: query: explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from
+(select A.key from A group by key) ss join
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Tez
+ Edges:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 5 <- Map 4 (SIMPLE_EDGE), Reducer 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: d1
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1
+ keys:
+ 0 key (type: string)
+ 1 key (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (1 = 1) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: d2
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: key (type: string)
+ sort order: +
+ Map-reduce partition columns: key (type: string)
+ Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string)
+ outputColumnNames: key
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: key (type: string)
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
+ Reducer 2
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Reducer 5
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0}
+ 1 {_col0}
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 31 Data size: 3196 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 31 Data size: 3196 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 31 Data size: 3196 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
+ limit: -1
+