Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id CBF8317D0C for ; Mon, 27 Apr 2015 20:54:57 +0000 (UTC) Received: (qmail 3236 invoked by uid 500); 27 Apr 2015 20:54:57 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 3063 invoked by uid 500); 27 Apr 2015 20:54:57 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 3015 invoked by uid 99); 27 Apr 2015 20:54:57 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 27 Apr 2015 20:54:57 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 727E9E17E4; Mon, 27 Apr 2015 20:54:57 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: hashutosh@apache.org To: commits@hive.apache.org Date: Mon, 27 Apr 2015 20:54:58 -0000 Message-Id: <81ddb8b282d74e7992d88fafa1721544@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/3] hive git commit: HIVE-10462 : CBO (Calcite Return Path): MapJoin and SMBJoin conversion not triggered (Jesus Camacho Rodriguez via Ashutosh Chauhan) HIVE-10462 : CBO (Calcite Return Path): MapJoin and SMBJoin conversion not triggered (Jesus Camacho Rodriguez via Ashutosh Chauhan) Signed-off-by: Ashutosh Chauhan Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e99e3c17 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e99e3c17 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e99e3c17 Branch: refs/heads/branch-1.2 Commit: e99e3c17046c36d82081545efc5dbad8ea243636 Parents: 71c7785 Author: Jesus Camacho Rodriguez Authored: Sun Apr 26 10:12:00 2015 -0700 Committer: Ashutosh Chauhan Committed: Mon Apr 27 13:52:36 2015 -0700 ---------------------------------------------------------------------- .../hadoop/hive/ql/optimizer/Optimizer.java | 7 + .../calcite/translator/HiveOpConverter.java | 1 - .../translator/HiveOpConverterPostProc.java | 134 ++ .../apache/hadoop/hive/ql/plan/JoinDesc.java | 12 + .../queries/clientpositive/cbo_rp_auto_join0.q | 24 + .../queries/clientpositive/cbo_rp_auto_join1.q | 274 ++++ .../clientpositive/cbo_rp_auto_join0.q.out | 258 +++ .../clientpositive/cbo_rp_auto_join1.q.out | 1512 ++++++++++++++++++ 8 files changed, 2221 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/e99e3c17/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java index 4ee4bdc..5afd4f2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -24,6 +24,7 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.optimizer.calcite.translator.HiveOpConverterPostProc; import org.apache.hadoop.hive.ql.optimizer.correlation.CorrelationOptimizer; import org.apache.hadoop.hive.ql.optimizer.correlation.ReduceSinkDeDuplication; import org.apache.hadoop.hive.ql.optimizer.index.RewriteGBUsingIndex; @@ -61,6 +62,12 @@ public class Optimizer { transformations = new ArrayList(); + // If we are translating Calcite operators into Hive operators, we need + // additional postprocessing + if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP)) { + transformations.add(new HiveOpConverterPostProc()); + } + // Add the transformation that computes the lineage information. transformations.add(new Generator()); if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)) { http://git-wip-us.apache.org/repos/asf/hive/blob/e99e3c17/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java index feb26bc..7311b71 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java @@ -94,7 +94,6 @@ import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.hive.ql.plan.UnionDesc; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; public class HiveOpConverter { http://git-wip-us.apache.org/repos/asf/hive/blob/e99e3c17/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java new file mode 100644 index 0000000..fcfe658 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.optimizer.calcite.translator; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.JoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; +import org.apache.hadoop.hive.ql.lib.Dispatcher; +import org.apache.hadoop.hive.ql.lib.ForwardWalker; +import org.apache.hadoop.hive.ql.lib.GraphWalker; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.lib.Rule; +import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.optimizer.Transform; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; + +public class HiveOpConverterPostProc implements Transform { + + private static final Log LOG = LogFactory.getLog(HiveOpConverterPostProc.class); + + private ParseContext pctx; + private Map> aliasToOpInfo; + private Map opToAlias; + private int uniqueCounter; + + @Override + public ParseContext transform(ParseContext pctx) throws SemanticException { + // 1. Initialize aux data structures + this.pctx = pctx; + this.aliasToOpInfo = new HashMap>(); + this.opToAlias = new HashMap(); + this.uniqueCounter = 0; + + // 2. Trigger transformation + Map opRules = new LinkedHashMap(); + opRules.put(new RuleRegExp("R1", JoinOperator.getOperatorName() + "%"), new JoinAnnotate()); + + Dispatcher disp = new DefaultRuleDispatcher(new DefaultAnnotate(), opRules, null); + GraphWalker ogw = new ForwardWalker(disp); + + List topNodes = new ArrayList(); + topNodes.addAll(pctx.getTopOps().values()); + ogw.startWalking(topNodes, null); + return pctx; + } + + private class JoinAnnotate implements NodeProcessor { + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + JoinOperator joinOp = (JoinOperator) nd; + joinOp.getName(); + + // 1. Additional data structures needed for the join optimization + // through Hive + String[] baseSrc = new String[joinOp.getParentOperators().size()]; + String[] rightAliases = new String[joinOp.getParentOperators().size()-1]; + for (int i = 0; i < joinOp.getParentOperators().size(); i++) { + ReduceSinkOperator rsOp = (ReduceSinkOperator) joinOp.getParentOperators().get(i); + final String opId = rsOp.getParentOperators().get(0).toString(); + baseSrc[i] = opToAlias.get(opId); + if (i == 0) { + joinOp.getConf().setLeftAlias(baseSrc[i]); + } else { + rightAliases[i-1] = baseSrc[i]; + } + } + joinOp.getConf().setBaseSrc(baseSrc); + joinOp.getConf().setRightAliases(rightAliases); + joinOp.getConf().setAliasToOpInfo(aliasToOpInfo); + + // 2. Generate self alias + final String joinOpAlias = genUniqueAlias(); + aliasToOpInfo.put(joinOpAlias, joinOp); + opToAlias.put(joinOp.toString(), joinOpAlias); + + // 3. Populate other data structures + pctx.getJoinOps().add(joinOp); + + return null; + } + + } + + private class DefaultAnnotate implements NodeProcessor { + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + Operator op = (Operator) nd; + + // 1. Generate self alias + final String opAlias = genUniqueAlias(); + aliasToOpInfo.put(opAlias, op); + opToAlias.put(op.toString(), opAlias); + + return null; + } + } + + private String genUniqueAlias() { + return "op-" + (++uniqueCounter); + } +} http://git-wip-us.apache.org/repos/asf/hive/blob/e99e3c17/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java index 1d3d1a2..37012b4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java @@ -543,6 +543,10 @@ public class JoinDesc extends AbstractOperatorDesc { return baseSrc; } + public void setBaseSrc(String[] baseSrc) { + this.baseSrc = baseSrc; + } + public String getId() { return id; } @@ -555,6 +559,10 @@ public class JoinDesc extends AbstractOperatorDesc { return aliasToOpInfo; } + public void setAliasToOpInfo(Map> aliasToOpInfo) { + this.aliasToOpInfo = aliasToOpInfo; + } + public boolean isLeftInputJoin() { return leftInputJoin; } @@ -571,6 +579,10 @@ public class JoinDesc extends AbstractOperatorDesc { return rightAliases; } + public void setRightAliases(String[] rightAliases) { + this.rightAliases = rightAliases; + } + public List getStreamAliases() { return streamAliases; } http://git-wip-us.apache.org/repos/asf/hive/blob/e99e3c17/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q ---------------------------------------------------------------------- diff --git a/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q b/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q new file mode 100644 index 0000000..ad9ebd1 --- /dev/null +++ b/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q @@ -0,0 +1,24 @@ +set hive.stats.fetch.column.stats=true; +set hive.auto.convert.join = true; + +explain +select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +SELECT cbo_t1.key as k1, cbo_t1.value as v1, + cbo_t2.key as k2, cbo_t2.value as v2 FROM + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t1 + JOIN + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t2 + SORT BY k1, v1, k2, v2 +) a; + +explain +select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +SELECT cbo_t1.key as k1, cbo_t1.value as v1, + cbo_t2.key as k2, cbo_t2.value as v2 FROM + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t1 + JOIN + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t2 + SORT BY k1, v1, k2, v2 +) a; http://git-wip-us.apache.org/repos/asf/hive/blob/e99e3c17/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q ---------------------------------------------------------------------- diff --git a/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q b/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q new file mode 100644 index 0000000..c1f4352 --- /dev/null +++ b/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q @@ -0,0 +1,274 @@ +set hive.stats.fetch.column.stats=true; +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +-- SORT_QUERY_RESULTS + +CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +insert overwrite table tbl1 +select * from src where key < 10; + +insert overwrite table tbl2 +select * from src where key < 10; + +analyze table tbl1 compute statistics; +analyze table tbl1 compute statistics for columns; + +analyze table tbl2 compute statistics; +analyze table tbl2 compute statistics for columns; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +set hive.auto.convert.sortmerge.join=true; + +-- The join is being performed as part of sub-query. It should be converted to a sort-merge join +explain +select count(*) from ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +) subq1; + +select count(*) from ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +) subq1; + +-- The join is being performed as part of more than one sub-query. It should be converted to a sort-merge join +explain +select count(*) from +( + select key, count(*) from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 + group by key +) subq2; + +select count(*) from +( + select key, count(*) from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 + group by key +) subq2; + +-- A join is being performed across different sub-queries, where a join is being performed in each of them. +-- Each sub-query should be converted to a sort-merge join. +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key; + +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join, although there is more than one level of sub-query +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key; + +-- Both the tables are nested sub-queries i.e more then 1 level of sub-query. +-- The join should be converted to a sort-merge join +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key; + +-- The subquery itself is being joined. Since the sub-query only contains selects and filters and the join key +-- is not getting modified, it should be converted to a sort-merge join. Note that the sub-query modifies one +-- item, but that is not part of the join key. +explain +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key; + +-- Since the join key is modified by the sub-query, neither sort-merge join not bucketized map-side +-- join should be performed +explain +select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; + +select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key; + +-- One of the tables is a sub-query and the other is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key; + +-- There are more than 2 inputs to the join, all of them being sub-queries. +-- It should be converted to to a sort-merge join +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on (subq1.key = subq2.key) + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key); + +-- The join is being performed on a nested sub-query, and an aggregation is performed after that. +-- The join should be converted to a sort-merge join +explain +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; + +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a; + +-- The join is followed by a multi-table insert. It should be converted to +-- a sort-merge join +explain select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key; + +select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key; + +-- The join is followed by a multi-table insert, and one of the inserts involves a reducer. +-- It should be converted to a sort-merge join +explain select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key; + +select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key; + http://git-wip-us.apache.org/repos/asf/hive/blob/e99e3c17/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out new file mode 100644 index 0000000..6fdc935 --- /dev/null +++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out @@ -0,0 +1,258 @@ +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 ( +SELECT cbo_t1.key as k1, cbo_t1.value as v1, + cbo_t2.key as k2, cbo_t2.value as v2 FROM + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t1 + JOIN + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t2 + SORT BY k1, v1, k2, v2 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +SELECT cbo_t1.key as k1, cbo_t1.value as v1, + cbo_t2.key as k2, cbo_t2.value as v2 FROM + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t1 + JOIN + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t2 + SORT BY k1, v1, k2, v2 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-6 is a root stage + Stage-2 depends on stages: Stage-6 + Stage-3 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-6 + Map Reduce Local Work + Alias -> Map Local Tables: + a:cbo_t1:cbo_t3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:cbo_t1:cbo_t3 + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE + HashTable Sink Operator + keys: + 0 + 1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 18 Data size: 6120 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 18 Data size: 6120 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + sort order: ++++ + Statistics: Num rows: 18 Data size: 6120 Basic stats: COMPLETE Column stats: COMPLETE + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 18 Data size: 6120 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(hash(_col0,_col1,_col2,_col3)) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Processor Tree: + ListSink + +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 ( +SELECT cbo_t1.key as k1, cbo_t1.value as v1, + cbo_t2.key as k2, cbo_t2.value as v2 FROM + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t1 + JOIN + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t2 + SORT BY k1, v1, k2, v2 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +select sum(hash(a.k1,a.v1,a.k2, a.v2)) +from ( +SELECT cbo_t1.key as k1, cbo_t1.value as v1, + cbo_t2.key as k2, cbo_t2.value as v2 FROM + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t1 + JOIN + (SELECT * FROM cbo_t3 WHERE cbo_t3.key < 10) cbo_t2 + SORT BY k1, v1, k2, v2 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-6 is a root stage + Stage-2 depends on stages: Stage-6 + Stage-3 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-6 + Map Reduce Local Work + Alias -> Map Local Tables: + a:cbo_t1:cbo_t3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:cbo_t1:cbo_t3 + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE + HashTable Sink Operator + keys: + 0 + 1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: cbo_t3 + Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 10) (type: boolean) + Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 6 Data size: 850 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 18 Data size: 6120 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 18 Data size: 6120 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + sort order: ++++ + Statistics: Num rows: 18 Data size: 6120 Basic stats: COMPLETE Column stats: COMPLETE + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 18 Data size: 6120 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(hash(_col0,_col1,_col2,_col3)) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Processor Tree: + ListSink + http://git-wip-us.apache.org/repos/asf/hive/blob/e99e3c17/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out new file mode 100644 index 0000000..2e23197 --- /dev/null +++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out @@ -0,0 +1,1512 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl1 +POSTHOOK: query: -- SORT_QUERY_RESULTS + +CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl1 +PREHOOK: query: CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl2 +POSTHOOK: query: CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl2 +PREHOOK: query: insert overwrite table tbl1 +select * from src where key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl1 +POSTHOOK: query: insert overwrite table tbl1 +select * from src where key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tbl1 +POSTHOOK: Lineage: tbl1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tbl1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: insert overwrite table tbl2 +select * from src where key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl2 +POSTHOOK: query: insert overwrite table tbl2 +select * from src where key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tbl2 +POSTHOOK: Lineage: tbl2.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tbl2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: analyze table tbl1 compute statistics +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Output: default@tbl1 +POSTHOOK: query: analyze table tbl1 compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Output: default@tbl1 +PREHOOK: query: analyze table tbl1 compute statistics for columns +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +#### A masked pattern was here #### +POSTHOOK: query: analyze table tbl1 compute statistics for columns +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +#### A masked pattern was here #### +PREHOOK: query: analyze table tbl2 compute statistics +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl2 +PREHOOK: Output: default@tbl2 +POSTHOOK: query: analyze table tbl2 compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl2 +POSTHOOK: Output: default@tbl2 +PREHOOK: query: analyze table tbl2 compute statistics for columns +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: analyze table tbl2 compute statistics for columns +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +PREHOOK: query: -- The join is being performed as part of sub-query. It should be converted to a sort-merge join +explain +select count(*) from ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +) subq1 +PREHOOK: type: QUERY +POSTHOOK: query: -- The join is being performed as part of sub-query. It should be converted to a sort-merge join +explain +select count(*) from ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +) subq1 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +) subq1 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +) subq1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +22 +PREHOOK: query: -- The join is being performed as part of more than one sub-query. It should be converted to a sort-merge join +explain +select count(*) from +( + select key, count(*) from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 + group by key +) subq2 +PREHOOK: type: QUERY +POSTHOOK: query: -- The join is being performed as part of more than one sub-query. It should be converted to a sort-merge join +explain +select count(*) from +( + select key, count(*) from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 + group by key +) subq2 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + 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: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from +( + select key, count(*) from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 + group by key +) subq2 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from +( + select key, count(*) from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 + group by key +) subq2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +6 +PREHOOK: query: -- A join is being performed across different sub-queries, where a join is being performed in each of them. +-- Each sub-query should be converted to a sort-merge join. +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key +PREHOOK: type: QUERY +POSTHOOK: query: -- A join is being performed across different sub-queries, where a join is being performed in each of them. +-- Each sub-query should be converted to a sort-merge join. +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-4 is a root stage + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Group By Operator + aggregations: count() + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + 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: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: bigint) + TableScan + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + 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 + + Stage: Stage-4 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Group By Operator + aggregations: count() + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +0 9 9 +2 1 1 +4 1 1 +5 9 9 +8 1 1 +9 1 1 +PREHOOK: query: -- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 6) (type: boolean) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +20 +PREHOOK: query: -- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join, although there is more than one level of sub-query +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The subquery itself is being joined. Since the sub-query only contains selects and filters, it should +-- be converted to a sort-merge join, although there is more than one level of sub-query +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join tbl2 b + on subq2.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +20 +PREHOOK: query: -- Both the tables are nested sub-queries i.e more then 1 level of sub-query. +-- The join should be converted to a sort-merge join +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Both the tables are nested sub-queries i.e more then 1 level of sub-query. +-- The join should be converted to a sort-merge join +explain +select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 + join + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq3 + where key < 6 + ) subq4 + on subq2.key = subq4.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +#### A masked pattern was here #### +20 +PREHOOK: query: -- The subquery itself is being joined. Since the sub-query only contains selects and filters and the join key +-- is not getting modified, it should be converted to a sort-merge join. Note that the sub-query modifies one +-- item, but that is not part of the join key. +explain +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The subquery itself is being joined. Since the sub-query only contains selects and filters and the join key +-- is not getting modified, it should be converted to a sort-merge join. Note that the sub-query modifies one +-- item, but that is not part of the join key. +explain +select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 8) (type: boolean) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from + (select a.key as key, concat(a.value, a.value) as value from tbl1 a where key < 8) subq1 + join + (select a.key as key, concat(a.value, a.value) as value from tbl2 a where key < 8) subq2 + on subq1.key = subq2.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +20 +PREHOOK: query: -- Since the join key is modified by the sub-query, neither sort-merge join not bucketized map-side +-- join should be performed +explain +select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since the join key is modified by the sub-query, neither sort-merge join not bucketized map-side +-- join should be performed +explain +select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: (key + 1) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: (key + 1) (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Statistics: Num rows: 10 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from + (select a.key +1 as key, concat(a.value, a.value) as value from tbl1 a) subq1 + join + (select a.key +1 as key, concat(a.value, a.value) as value from tbl2 a) subq2 + on subq1.key = subq2.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +22 +PREHOOK: query: -- One of the tables is a sub-query and the other is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key +PREHOOK: type: QUERY +POSTHOOK: query: -- One of the tables is a sub-query and the other is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 6) (type: boolean) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join tbl2 a on subq1.key = a.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +20 +PREHOOK: query: -- There are more than 2 inputs to the join, all of them being sub-queries. +-- It should be converted to to a sort-merge join +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on (subq1.key = subq2.key) + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- There are more than 2 inputs to the join, all of them being sub-queries. +-- It should be converted to to a sort-merge join +explain +select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on (subq1.key = subq2.key) + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 6) (type: boolean) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (key < 6) (type: boolean) + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + 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: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from + (select a.key as key, a.value as value from tbl1 a where key < 6) subq1 + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq2 + on subq1.key = subq2.key + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq3 + on (subq1.key = subq3.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +56 +PREHOOK: query: -- The join is being performed on a nested sub-query, and an aggregation is performed after that. +-- The join should be converted to a sort-merge join +explain +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a +PREHOOK: type: QUERY +POSTHOOK: query: -- The join is being performed on a nested sub-query, and an aggregation is performed after that. +-- The join should be converted to a sort-merge join +explain +select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from ( + select subq2.key as key, subq2.value as value1, b.value as value2 from + ( + select * from + ( + select a.key as key, a.value as value from tbl1 a where key < 8 + ) subq1 + where key < 6 + ) subq2 +join tbl2 b +on subq2.key = b.key) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +20 +PREHOOK: query: -- The join is followed by a multi-table insert. It should be converted to +-- a sort-merge join +explain select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The join is followed by a multi-table insert. It should be converted to +-- a sort-merge join +explain select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1, _col2 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +2 val_2 val_2 +4 val_4 val_4 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +8 val_8 val_8 +9 val_9 val_9 +PREHOOK: query: -- The join is followed by a multi-table insert, and one of the inserts involves a reducer. +-- It should be converted to a sort-merge join +explain select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The join is followed by a multi-table insert, and one of the inserts involves a reducer. +-- It should be converted to a sort-merge join +explain select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1, _col2 + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +2 val_2 val_2 +4 val_4 val_4 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +5 val_5 val_5 +8 val_8 val_8 +9 val_9 val_9