diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index 558dd02..dc4a04f 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -410,6 +410,22 @@ spark.query.files=add_part_multiple.q \ alter_merge_stats_orc.q \ avro_compression_enabled_native.q \ avro_decimal_native.q \ + auto_sortmerge_join_1.q \ + auto_sortmerge_join_2.q \ + auto_sortmerge_join_3.q \ + auto_sortmerge_join_4.q \ + auto_sortmerge_join_5.q \ + auto_sortmerge_join_6.q \ + auto_sortmerge_join_7.q \ + auto_sortmerge_join_8.q \ + auto_sortmerge_join_9.q \ + auto_sortmerge_join_10.q \ + auto_sortmerge_join_11.q \ + auto_sortmerge_join_12.q \ + auto_sortmerge_join_13.q \ + auto_sortmerge_join_14.q \ + auto_sortmerge_join_15.q \ + auto_sortmerge_join_16.q \ bucket2.q \ bucket3.q \ bucket4.q \ @@ -535,6 +551,7 @@ spark.query.files=add_part_multiple.q \ script_env_var2.q \ script_pipe.q \ scriptfile1.q \ + smb_mapjoin_17.q \ sort.q \ spark_test.q \ subquery_multiinsert.q \ @@ -625,5 +642,4 @@ spark.query.files=add_part_multiple.q \ windowing.q \ subquery_exists.q \ subquery_in.q \ - auto_sortmerge_join_1.q \ skewjoin.q diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java index 5b4c74c..f866bda 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -96,7 +96,7 @@ public void initialize(HiveConf hiveConf) { // If optimize hive.optimize.bucketmapjoin.sortedmerge is set, add both // BucketMapJoinOptimizer and SortedMergeBucketMapJoinOptimizer if ((HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)) - && !isTezExecEngine && !isSparkExecEngine) { + && !isTezExecEngine) { if (!bucketMapJoinOptimizer) { // No need to add BucketMapJoinOptimizer twice transformations.add(new BucketMapJoinOptimizer()); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinFactory.java new file mode 100644 index 0000000..4eb9fca --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinFactory.java @@ -0,0 +1,196 @@ +/** +* Licensed to the Apache Software Foundation (ASF) under one +* or more contributor license agreements. See the NOTICE file +* distributed with this work for additional information +* regarding copyright ownership. The ASF licenses this file +* to you under the Apache License, Version 2.0 (the +* "License"); you may not use this file except in compliance +* with the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.hadoop.hive.ql.optimizer.spark; + +import org.apache.hadoop.hive.ql.exec.*; +import org.apache.hadoop.hive.ql.exec.spark.SparkTask; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext; +import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx; +import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.spark.GenSparkProcContext; +import org.apache.hadoop.hive.ql.plan.*; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; + +/** +* Operator factory for MapJoin processing. +*/ +public final class SparkMapJoinFactory { + + public static int getPositionParent(AbstractMapJoinOperator op, + Stack stack) { + int pos = 0; + int size = stack.size(); + assert size >= 2 && stack.get(size - 1) == op; + Operator parent = + (Operator) stack.get(size - 2); + List> parOp = op.getParentOperators(); + pos = parOp.indexOf(parent); + assert pos < parOp.size(); + return pos; + } + + /** + * MapJoin processor. + * The user can specify a mapjoin hint to specify that the input should be processed as a + * mapjoin instead of map-reduce join. If hive.auto.convert.join is set to true, the + * user need not specify the hint explicitly, but hive will automatically process the joins + * as a mapjoin whenever possible. However, a join can only be processed as a bucketized + * map-side join or a sort merge join, if the user has provided the hint explicitly. This + * will be fixed as part of HIVE-3433, and eventually, we should remove support for mapjoin + * hint. + * However, currently, the mapjoin hint is processed as follows: + * A mapjoin will have 'n' parents for a n-way mapjoin, and therefore the mapjoin operator + * will be encountered 'n' times (one for each parent). Since a reduceSink operator is not + * allowed before a mapjoin, the task for the mapjoin will always be a root task. The task + * corresponding to the mapjoin is converted to a root task when the operator is encountered + * for the first time. When the operator is encountered subsequently, the current task is + * merged with the root task for the mapjoin. Note that, it is possible that the map-join task + * may be performed as a bucketized map-side join (or sort-merge join), the map join operator + * is enhanced to contain the bucketing info. when it is encountered. + */ + private static class TableScanMapJoinProcessor implements NodeProcessor { + + public static void setupBucketMapJoinInfo(MapWork plan, + AbstractMapJoinOperator currMapJoinOp) { + if (currMapJoinOp != null) { + Map>> aliasBucketFileNameMapping = + currMapJoinOp.getConf().getAliasBucketFileNameMapping(); + if (aliasBucketFileNameMapping != null) { + MapredLocalWork localPlan = plan.getMapLocalWork(); + if (localPlan == null) { + if (currMapJoinOp instanceof SMBMapJoinOperator) { + localPlan = ((SMBMapJoinOperator) currMapJoinOp).getConf().getLocalWork(); + } + } else { + // local plan is not null, we want to merge it into SMBMapJoinOperator's local work + if (currMapJoinOp instanceof SMBMapJoinOperator) { + MapredLocalWork smbLocalWork = ((SMBMapJoinOperator) currMapJoinOp).getConf() + .getLocalWork(); + if (smbLocalWork != null) { + localPlan.getAliasToFetchWork().putAll(smbLocalWork.getAliasToFetchWork()); + localPlan.getAliasToWork().putAll(smbLocalWork.getAliasToWork()); + } + } + } + + if (localPlan == null) { + return; + } + + if (currMapJoinOp instanceof SMBMapJoinOperator) { + plan.setMapLocalWork(null); + ((SMBMapJoinOperator) currMapJoinOp).getConf().setLocalWork(localPlan); + } else { + plan.setMapLocalWork(localPlan); + } + BucketMapJoinContext bucketMJCxt = new BucketMapJoinContext(); + localPlan.setBucketMapjoinContext(bucketMJCxt); + bucketMJCxt.setAliasBucketFileNameMapping(aliasBucketFileNameMapping); + bucketMJCxt.setBucketFileNameMapping( + currMapJoinOp.getConf().getBigTableBucketNumMapping()); + localPlan.setInputFileChangeSensitive(true); + bucketMJCxt.setMapJoinBigTableAlias(currMapJoinOp.getConf().getBigTableAlias()); + bucketMJCxt + .setBucketMatcherClass(org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class); + bucketMJCxt.setBigTablePartSpecToFileMapping( + currMapJoinOp.getConf().getBigTablePartSpecToFileMapping()); + // BucketizedHiveInputFormat should be used for either sort merge join or bucket map join + if ((currMapJoinOp instanceof SMBMapJoinOperator) + || (currMapJoinOp.getConf().isBucketMapJoin())) { + plan.setUseBucketizedHiveInputFormat(true); + } + } + } + } + + /** + * Initialize the current plan by adding it to root tasks. Since a reduce sink + * cannot be present before a mapjoin, and the mapjoin operator is encountered + * for the first time, the task corresposding to the mapjoin is added to the + * root tasks. + * + * @param opProcCtx + * processing context + */ + private static void initMapJoinPlan(MapWork mapWork, + GenSparkProcContext opProcCtx, boolean local) + throws SemanticException { + TableScanOperator ts = (TableScanOperator) opProcCtx.currentRootOperator; + String currAliasId = findAliasId(opProcCtx, ts); + GenMapRedUtils.setMapWork(mapWork, opProcCtx.parseContext, + opProcCtx.inputs, null, ts, currAliasId, opProcCtx.conf, local); + } + + private static String findAliasId(GenSparkProcContext opProcCtx, TableScanOperator ts) { + for (String alias : opProcCtx.topOps.keySet()) { + if (opProcCtx.topOps.get(alias) == ts) { + return alias; + } + } + return null; + } + + /* + * The mapjoin operator will be encountered many times (n times for a n-way join). Since a + * reduceSink operator is not allowed before a mapjoin, the task for the mapjoin will always + * be a root task. The task corresponding to the mapjoin is converted to a root task when the + * operator is encountered for the first time. When the operator is encountered subsequently, + * the current task is merged with the root task for the mapjoin. Note that, it is possible + * that the map-join task may be performed as a bucketized map-side join (or sort-merge join), + * the map join operator is enhanced to contain the bucketing info. when it is encountered. + */ + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + AbstractMapJoinOperator mapJoin = (AbstractMapJoinOperator) nd; + GenSparkProcContext ctx = (GenSparkProcContext) procCtx; + + SparkTask currTask = ctx.currentTask; + + // find the branch on which this processor was invoked + int pos = getPositionParent(mapJoin, stack); + boolean local = pos != mapJoin.getConf().getPosBigTable(); + + MapWork mapWork = ctx.smbJoinWorkMap.get(mapJoin); + initMapJoinPlan(mapWork, ctx, local); + + // find the associated mapWork that contains this processor. + setupBucketMapJoinInfo(mapWork, mapJoin); + + // local aliases need not to hand over context further + return false; + } + } + + public static NodeProcessor getTableScanMapJoin() { + return new TableScanMapJoinProcessor(); + } + + private SparkMapJoinFactory() { + // prevent instantiation + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java index d7744e9..d293ddf 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java @@ -19,34 +19,16 @@ package org.apache.hadoop.hive.ql.parse.spark; import java.io.Serializable; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.DependencyCollectionTask; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.UnionOperator; +import org.apache.hadoop.hive.ql.exec.*; import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.DependencyCollectionWork; -import org.apache.hadoop.hive.ql.plan.MoveWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; +import org.apache.hadoop.hive.ql.plan.*; /** * GenSparkProcContext maintains information about the tasks and operators @@ -117,6 +99,9 @@ // map that says which mapjoin belongs to which work item public final Map> mapJoinWorkMap; + // a map to keep track of which MapWork item holds which SMBMapJoinOp + public final Map smbJoinWorkMap; + // a map to keep track of which root generated which work public final Map, BaseWork> rootToWorkMap; @@ -147,16 +132,21 @@ // remember which reducesinks we've already connected public final Set connectedReduceSinks; + // Alias to operator map, from the semantic analyzer. + // This is necessary as sometimes semantic analyzer's mapping is different than operator's own alias. + public final Map> topOps; + @SuppressWarnings("unchecked") public GenSparkProcContext(HiveConf conf, ParseContext parseContext, List> moveTask, List> rootTasks, - Set inputs, Set outputs) { + Set inputs, Set outputs, Map> topOps) { this.conf = conf; this.parseContext = parseContext; this.moveTask = moveTask; this.rootTasks = rootTasks; this.inputs = inputs; this.outputs = outputs; + this.topOps = topOps; this.defaultTask = (SparkTask) TaskFactory.get( new SparkWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID)), conf); this.rootTasks.add(defaultTask); @@ -164,6 +154,7 @@ public GenSparkProcContext(HiveConf conf, ParseContext parseContext, this.leafOperatorToFollowingWork = new LinkedHashMap, BaseWork>(); this.linkOpWithWorkMap = new LinkedHashMap, Map>(); this.linkWorkWithReduceSinkMap = new LinkedHashMap>(); + this.smbJoinWorkMap = new LinkedHashMap(); this.mapJoinWorkMap = new LinkedHashMap>(); this.rootToWorkMap = new LinkedHashMap, BaseWork>(); this.childToWorkMap = new LinkedHashMap, List>(); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index 280edde..cb25925 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java @@ -86,7 +86,7 @@ public UnionWork createUnionWork(GenSparkProcContext context, Operator operat return unionWork; } - public ReduceWork createReduceWork(GenSparkProcContext context, Operator root, SparkWork sparkWork) { + public ReduceWork createReduceWork(GenSparkProcContext context, Operator root, SparkWork sparkWork) throws SemanticException { Preconditions.checkArgument(!root.getParentOperators().isEmpty(), "AssertionError: expected root.getParentOperators() to be non-empty"); @@ -120,10 +120,15 @@ public ReduceWork createReduceWork(GenSparkProcContext context, Operator root } if (reduceWork.getReducer() instanceof JoinOperator) { - //reduce-side join + //reduce-side join, use MR-style shuffle + edgeProp.setMRShuffle(); + } + if (getChildOperator(reduceWork.getReducer(), FileSinkOperator.class) != null) { + //FileSink to bucketed files assume hash partitioning, so also use MR-style shuffle. edgeProp.setMRShuffle(); } + sparkWork.connect( context.preceedingWork, reduceWork, edgeProp); @@ -150,7 +155,12 @@ protected void setupReduceSink(GenSparkProcContext context, ReduceWork reduceWor } public MapWork createMapWork(GenSparkProcContext context, Operator root, - SparkWork sparkWork, PrunedPartitionList partitions) throws SemanticException { + SparkWork sparkWork, PrunedPartitionList partitions) throws SemanticException { + return createMapWork(context, root, sparkWork, partitions, false); + } + + public MapWork createMapWork(GenSparkProcContext context, Operator root, + SparkWork sparkWork, PrunedPartitionList partitions, boolean deferSetup) throws SemanticException { Preconditions.checkArgument(root.getParentOperators().isEmpty(), "AssertionError: expected root.getParentOperators() to be empty"); MapWork mapWork = new MapWork("Map "+ (++sequenceNumber)); @@ -162,7 +172,9 @@ public MapWork createMapWork(GenSparkProcContext context, Operator root, root.getClass().getName()); String alias = ((TableScanOperator)root).getConf().getAlias(); - setupMapWork(mapWork, context, partitions, root, alias); + if (!deferSetup) { + setupMapWork(mapWork, context, partitions, root, alias); + } // add new item to the Spark work sparkWork.add(mapWork); @@ -327,4 +339,26 @@ public static boolean isSortNecessary(ReduceSinkOperator reduceSinkOperator) { } return true; } + + /** + * Is an operator of the given class a child of the given operator. This is more flexible + * than GraphWalker to tell apart subclasses such as SMBMapJoinOp vs MapJoinOp that have a common name. + * @param op parent operator to start search + * @param klazz given class + * @return + * @throws SemanticException + */ + public static Operator getChildOperator(Operator op, Class klazz) throws SemanticException { + if (klazz.isInstance(op)) { + return op; + } + List> childOperators = op.getChildOperators(); + for (Operator childOp : childOperators) { + Operator result = getChildOperator(childOp, klazz); + if (result != null) { + return result; + } + } + return null; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java index ac94ea0..ee2d16e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java @@ -18,21 +18,15 @@ package org.apache.hadoop.hive.ql.parse.spark; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.Map.Entry; -import java.util.Stack; import com.google.common.base.Strings; 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.exec.*; -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.*; import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.*; @@ -96,6 +90,26 @@ public Object process(Node nd, Stack stack, SparkWork sparkWork = context.currentTask.getWork(); + + if (GenSparkUtils.getChildOperator(root, DummyStoreOperator.class) != null) { + /* + * SMB join case: + * + * (Big) (Small) (Small) + * TS TS TS + * \ | / + * \ DS DS + * \ | / + * SMBJoinOP + * + * Only create MapWork rooted at TS of big table. + * If there are dummy-store operators anywhere in TS's children path, then this is for the small tables. + * No separate Map-Task need to be created for small table TS, as they will be read by the MapWork of the big-table. + */ + return null; + } + SMBMapJoinOperator smbOp = (SMBMapJoinOperator) GenSparkUtils.getChildOperator(root, SMBMapJoinOperator.class); + // Right now the work graph is pretty simple. If there is no // Preceding work we have a root and will generate a map // vertex. If there is a preceding work we will generate @@ -113,7 +127,18 @@ public Object process(Node nd, Stack stack, } else { // create a new vertex if (context.preceedingWork == null) { - work = utils.createMapWork(context, root, sparkWork, null); + if (smbOp != null) { + //This logic is for SortMergeBucket MapJoin case. + //This MapWork (of big-table, see above..) is later initialized by SparkMapJoinFactory processor, so don't initialize it here. + //Just keep track of it in the context, for later processing. + work = utils.createMapWork(context, root, sparkWork, null, true); + if (context.smbJoinWorkMap.get(smbOp) != null) { + throw new SemanticException("Each SMBMapJoin should be associated only with one Mapwork"); + } + context.smbJoinWorkMap.put(smbOp, (MapWork) work); + } else { + work = utils.createMapWork(context, root, sparkWork, null); + } } else { work = utils.createReduceWork(context, root, sparkWork); } @@ -278,7 +303,11 @@ public Object process(Node nd, Stack stack, edgeProp.setShuffleSort(); } if (rWork.getReducer() instanceof JoinOperator) { - //reduce-side join + //reduce-side join, use MR-style shuffle + edgeProp.setMRShuffle(); + } + if (GenSparkUtils.getChildOperator(rWork.getReducer(), FileSinkOperator.class) != null) { + //FileSink to bucketed files assume hash partitioning, so also use MR-style shuffle. edgeProp.setMRShuffle(); } sparkWork.connect(work, rWork, edgeProp); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java index 644c681..e728106 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java @@ -32,14 +32,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -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.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.UnionOperator; +import org.apache.hadoop.hive.ql.exec.*; import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; @@ -54,12 +47,14 @@ import org.apache.hadoop.hive.ql.lib.Rule; import org.apache.hadoop.hive.ql.lib.RuleRegExp; import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.optimizer.MapJoinFactory; import org.apache.hadoop.hive.ql.optimizer.physical.CrossProductCheck; import org.apache.hadoop.hive.ql.optimizer.physical.NullScanOptimizer; import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger; import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer; import org.apache.hadoop.hive.ql.optimizer.spark.SetSparkReducerParallelism; +import org.apache.hadoop.hive.ql.optimizer.spark.SparkMapJoinFactory; import org.apache.hadoop.hive.ql.parse.GlobalLimitCtx; import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.SemanticException; @@ -137,7 +132,7 @@ protected void generateTaskTree(List> rootTasks, Pa GenSparkWork genSparkWork = new GenSparkWork(GenSparkUtils.getUtils()); GenSparkProcContext procCtx = new GenSparkProcContext( - conf, tempParseContext, mvTask, rootTasks, inputs, outputs); + conf, tempParseContext, mvTask, rootTasks, inputs, outputs, pCtx.getTopOps()); // -------------------- First Pass --------------------- @@ -159,6 +154,7 @@ protected void generateTaskTree(List> rootTasks, Pa new SparkMergeTaskProcessor()); opRules.put(new RuleRegExp("Union", UnionOperator.getOperatorName() + "%"), new SparkMergeTaskProcessor()); + disp = new DefaultRuleDispatcher(null, opRules, procCtx); topNodes = new ArrayList(); topNodes.addAll(procCtx.tempTS); // First process temp TS @@ -220,6 +216,22 @@ public Object process(Node n, Stack s, ogw = new GenSparkWorkWalker(disp, procCtx); ogw.startWalking(topNodes, null); + + + // ------------------- Fourth Pass ----------------------- + // SMB Join optimizations. Depend on SparkWork already generated. + opRules.clear(); + opRules.put(new RuleRegExp("MapJoin", MapJoinOperator.getOperatorName() + "%"), + SparkMapJoinFactory.getTableScanMapJoin()); + + + disp = new DefaultRuleDispatcher(null, opRules, procCtx); + topNodes = new ArrayList(); + topNodes.addAll(pCtx.getTopOps().values()); + topNodes.addAll(procCtx.tempTS); + ogw = new GenSparkWorkWalker(disp, procCtx); + ogw.startWalking(topNodes, null); + // we need to clone some operator plans and remove union operators still for (BaseWork w: procCtx.workWithUnionOperators) { GenSparkUtils.getUtils().removeUnionOperators(conf, procCtx, w); diff --git ql/src/test/results/clientpositive/spark/auto_join32.q.out ql/src/test/results/clientpositive/spark/auto_join32.q.out new file mode 100644 index 0000000..18b90ce --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_join32.q.out @@ -0,0 +1,516 @@ +PREHOOK: query: -- empty tables +create table studenttab10k (name string, age int, gpa double) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@studenttab10k +POSTHOOK: query: -- empty tables +create table studenttab10k (name string, age int, gpa double) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@studenttab10k +PREHOOK: query: create table votertab10k (name string, age int, registration string, contributions float) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@votertab10k +POSTHOOK: query: create table votertab10k (name string, age int, registration string, contributions float) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@votertab10k +PREHOOK: query: explain select s.name, count(distinct registration) +from studenttab10k s join votertab10k v +on (s.name = v.name) +group by s.name +PREHOOK: type: QUERY +POSTHOOK: query: explain select s.name, count(distinct registration) +from studenttab10k s join votertab10k v +on (s.name = v.name) +group by s.name +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT), Map 4 (GROUP PARTITION-LEVEL SORT) + Reducer 3 <- Reducer 2 (GROUP SORT) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: v + Filter Operator + predicate: name is not null (type: boolean) + Reduce Output Operator + key expressions: name (type: string) + sort order: + + Map-reduce partition columns: name (type: string) + value expressions: registration (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: s + Filter Operator + predicate: name is not null (type: boolean) + Reduce Output Operator + key expressions: name (type: string) + sort order: + + Map-reduce partition columns: name (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {KEY.reducesinkkey0} + 1 {VALUE._col1} + outputColumnNames: _col0, _col8 + Select Operator + expressions: _col0 (type: string), _col8 (type: string) + outputColumnNames: _col0, _col8 + Group By Operator + aggregations: count(DISTINCT _col8) + keys: _col0 (type: string), _col8 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select s.name, count(distinct registration) +from studenttab10k s join votertab10k v +on (s.name = v.name) +group by s.name +PREHOOK: type: QUERY +PREHOOK: Input: default@studenttab10k +PREHOOK: Input: default@votertab10k +#### A masked pattern was here #### +POSTHOOK: query: select s.name, count(distinct registration) +from studenttab10k s join votertab10k v +on (s.name = v.name) +group by s.name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@studenttab10k +POSTHOOK: Input: default@votertab10k +#### A masked pattern was here #### +PREHOOK: query: -- smb +create table studenttab10k_smb (name string, age int, gpa double) clustered by (name) sorted by (name) into 2 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@studenttab10k_smb +POSTHOOK: query: -- smb +create table studenttab10k_smb (name string, age int, gpa double) clustered by (name) sorted by (name) into 2 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@studenttab10k_smb +PREHOOK: query: create table votertab10k_smb (name string, age int, registration string, contributions float) clustered by (name) sorted by (name) into 2 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@votertab10k_smb +POSTHOOK: query: create table votertab10k_smb (name string, age int, registration string, contributions float) clustered by (name) sorted by (name) into 2 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@votertab10k_smb +PREHOOK: query: explain select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name +PREHOOK: type: QUERY +POSTHOOK: query: explain select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP SORT) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: s + Filter Operator + predicate: name is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {name} + 1 {registration} + keys: + 0 name (type: string) + 1 name (type: string) + outputColumnNames: _col0, _col8 + Select Operator + expressions: _col0 (type: string), _col8 (type: string) + outputColumnNames: _col0, _col8 + Group By Operator + aggregations: count(DISTINCT _col8) + keys: _col0 (type: string), _col8 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name +PREHOOK: type: QUERY +PREHOOK: Input: default@studenttab10k_smb +PREHOOK: Input: default@votertab10k_smb +#### A masked pattern was here #### +POSTHOOK: query: select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@studenttab10k_smb +POSTHOOK: Input: default@votertab10k_smb +#### A masked pattern was here #### +PREHOOK: query: load data local inpath '../../data/files/empty1.txt' into table studenttab10k_smb +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@studenttab10k_smb +POSTHOOK: query: load data local inpath '../../data/files/empty1.txt' into table studenttab10k_smb +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@studenttab10k_smb +PREHOOK: query: load data local inpath '../../data/files/empty2.txt' into table studenttab10k_smb +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@studenttab10k_smb +POSTHOOK: query: load data local inpath '../../data/files/empty2.txt' into table studenttab10k_smb +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@studenttab10k_smb +PREHOOK: query: load data local inpath '../../data/files/empty1.txt' into table votertab10k_smb +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@votertab10k_smb +POSTHOOK: query: load data local inpath '../../data/files/empty1.txt' into table votertab10k_smb +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@votertab10k_smb +PREHOOK: query: load data local inpath '../../data/files/empty2.txt' into table votertab10k_smb +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@votertab10k_smb +POSTHOOK: query: load data local inpath '../../data/files/empty2.txt' into table votertab10k_smb +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@votertab10k_smb +PREHOOK: query: explain select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name +PREHOOK: type: QUERY +POSTHOOK: query: explain select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP SORT) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: s + Filter Operator + predicate: name is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {name} + 1 {registration} + keys: + 0 name (type: string) + 1 name (type: string) + outputColumnNames: _col0, _col8 + Select Operator + expressions: _col0 (type: string), _col8 (type: string) + outputColumnNames: _col0, _col8 + Group By Operator + aggregations: count(DISTINCT _col8) + keys: _col0 (type: string), _col8 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name +PREHOOK: type: QUERY +PREHOOK: Input: default@studenttab10k_smb +PREHOOK: Input: default@votertab10k_smb +#### A masked pattern was here #### +POSTHOOK: query: select s.name, count(distinct registration) +from studenttab10k_smb s join votertab10k_smb v +on (s.name = v.name) +group by s.name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@studenttab10k_smb +POSTHOOK: Input: default@votertab10k_smb +#### A masked pattern was here #### +PREHOOK: query: -- smb + partitions +create table studenttab10k_part (name string, age int, gpa double) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@studenttab10k_part +POSTHOOK: query: -- smb + partitions +create table studenttab10k_part (name string, age int, gpa double) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@studenttab10k_part +PREHOOK: query: create table votertab10k_part (name string, age int, registration string, contributions float) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@votertab10k_part +POSTHOOK: query: create table votertab10k_part (name string, age int, registration string, contributions float) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@votertab10k_part +PREHOOK: query: load data local inpath '../../data/files/empty1.txt' into table studenttab10k_part partition (p='foo') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@studenttab10k_part +POSTHOOK: query: load data local inpath '../../data/files/empty1.txt' into table studenttab10k_part partition (p='foo') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@studenttab10k_part +POSTHOOK: Output: default@studenttab10k_part@p=foo +PREHOOK: query: load data local inpath '../../data/files/empty2.txt' into table studenttab10k_part partition (p='foo') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@studenttab10k_part@p=foo +POSTHOOK: query: load data local inpath '../../data/files/empty2.txt' into table studenttab10k_part partition (p='foo') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@studenttab10k_part@p=foo +PREHOOK: query: load data local inpath '../../data/files/empty1.txt' into table votertab10k_part partition (p='foo') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@votertab10k_part +POSTHOOK: query: load data local inpath '../../data/files/empty1.txt' into table votertab10k_part partition (p='foo') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@votertab10k_part +POSTHOOK: Output: default@votertab10k_part@p=foo +PREHOOK: query: load data local inpath '../../data/files/empty2.txt' into table votertab10k_part partition (p='foo') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@votertab10k_part@p=foo +POSTHOOK: query: load data local inpath '../../data/files/empty2.txt' into table votertab10k_part partition (p='foo') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@votertab10k_part@p=foo +PREHOOK: query: explain select s.name, count(distinct registration) +from studenttab10k_part s join votertab10k_part v +on (s.name = v.name) +where s.p = 'bar' +and v.p = 'bar' +group by s.name +PREHOOK: type: QUERY +POSTHOOK: query: explain select s.name, count(distinct registration) +from studenttab10k_part s join votertab10k_part v +on (s.name = v.name) +where s.p = 'bar' +and v.p = 'bar' +group by s.name +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP SORT) +#### A masked pattern was here #### + Vertices: + Map 1 + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(DISTINCT KEY._col1:0._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select s.name, count(distinct registration) +from studenttab10k_part s join votertab10k_part v +on (s.name = v.name) +where s.p = 'bar' +and v.p = 'bar' +group by s.name +PREHOOK: type: QUERY +PREHOOK: Input: default@studenttab10k_part +PREHOOK: Input: default@votertab10k_part +#### A masked pattern was here #### +POSTHOOK: query: select s.name, count(distinct registration) +from studenttab10k_part s join votertab10k_part v +on (s.name = v.name) +where s.p = 'bar' +and v.p = 'bar' +group by s.name +POSTHOOK: type: QUERY +POSTHOOK: Input: default@studenttab10k_part +POSTHOOK: Input: default@votertab10k_part +#### A masked pattern was here #### +PREHOOK: query: drop table studenttab10k +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@studenttab10k +PREHOOK: Output: default@studenttab10k +POSTHOOK: query: drop table studenttab10k +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@studenttab10k +POSTHOOK: Output: default@studenttab10k +PREHOOK: query: drop table votertab10k +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@votertab10k +PREHOOK: Output: default@votertab10k +POSTHOOK: query: drop table votertab10k +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@votertab10k +POSTHOOK: Output: default@votertab10k +PREHOOK: query: drop table studenttab10k_smb +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@studenttab10k_smb +PREHOOK: Output: default@studenttab10k_smb +POSTHOOK: query: drop table studenttab10k_smb +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@studenttab10k_smb +POSTHOOK: Output: default@studenttab10k_smb +PREHOOK: query: drop table votertab10k_smb +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@votertab10k_smb +PREHOOK: Output: default@votertab10k_smb +POSTHOOK: query: drop table votertab10k_smb +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@votertab10k_smb +POSTHOOK: Output: default@votertab10k_smb +PREHOOK: query: drop table studenttab10k_part +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@studenttab10k_part +PREHOOK: Output: default@studenttab10k_part +POSTHOOK: query: drop table studenttab10k_part +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@studenttab10k_part +POSTHOOK: Output: default@studenttab10k_part +PREHOOK: query: drop table votertab10k_part +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@votertab10k_part +PREHOOK: Output: default@votertab10k_part +POSTHOOK: query: drop table votertab10k_part +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@votertab10k_part +POSTHOOK: Output: default@votertab10k_part diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out index 08e0406..a06f59d 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out @@ -149,8 +149,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -161,12 +160,26 @@ STAGE PLANS: Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - tag: 1 - auto parallelism: true + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -269,93 +282,7 @@ STAGE PLANS: Truncated Path -> Alias: /bucket_big/ds=2008-04-08 [b] /bucket_big/ds=2008-04-09 [b] - Map 4 - Map Operator Tree: - TableScan - alias: a - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - tag: 0 - auto parallelism: true - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [a] Reducer 2 - Needs Tagging: true - Reduce Operator Tree: - Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 - 1 - Select Operator - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: _col0 (type: bigint) - auto parallelism: false - Reducer 3 Needs Tagging: false Reduce Operator Tree: Group By Operator @@ -453,91 +380,37 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan - alias: b - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - tag: 1 - auto parallelism: true - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] - Map 4 - Map Operator Tree: - TableScan alias: a GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - tag: 0 - auto parallelism: true + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -641,25 +514,6 @@ STAGE PLANS: /bucket_big/ds=2008-04-08 [a] /bucket_big/ds=2008-04-09 [a] Reducer 2 - Needs Tagging: true - Reduce Operator Tree: - Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 - 1 - Select Operator - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: _col0 (type: bigint) - auto parallelism: false - Reducer 3 Needs Tagging: false Reduce Operator Tree: Group By Operator @@ -757,91 +611,37 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) - Reducer 3 <- Reducer 2 (GROUP, 1) + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 Map Operator Tree: TableScan - alias: b - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - tag: 1 - auto parallelism: true - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] - Map 4 - Map Operator Tree: - TableScan alias: a GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - tag: 0 - auto parallelism: true + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -945,25 +745,6 @@ STAGE PLANS: /bucket_big/ds=2008-04-08 [a] /bucket_big/ds=2008-04-09 [a] Reducer 2 - Needs Tagging: true - Reduce Operator Tree: - Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 - 1 - Select Operator - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: _col0 (type: bigint) - auto parallelism: false - Reducer 3 Needs Tagging: false Reduce Operator Tree: Group By Operator diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out new file mode 100644 index 0000000..197150e --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out @@ -0,0 +1,328 @@ +PREHOOK: query: 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: 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 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +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 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +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: -- One of the subqueries contains a union, so it should not 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 < 6 + union all + select a.key as key, a.value as value from tbl1 a where key < 6 + ) usubq1 ) 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: -- One of the subqueries contains a union, so it should not 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 < 6 + union all + select a.key as key, a.value as value from tbl1 a where key < 6 + ) usubq1 ) 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 + Spark + Edges: + Reducer 3 <- Map 6 (GROUP PARTITION-LEVEL SORT, 1), Union 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (GROUP PARTITION-LEVEL SORT, 1) + Union 2 <- Map 1 (NONE, 0), Map 5 (NONE, 0) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Union 2 + Vertex: Union 2 + + 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 < 6 + union all + select a.key as key, a.value as value from tbl1 a where key < 6 + ) usubq1 ) 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 * from + (select a.key as key, a.value as value from tbl1 a where key < 6 + union all + select a.key as key, a.value as value from tbl1 a where key < 6 + ) usubq1 ) 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 #### +40 +PREHOOK: query: -- One of the subqueries contains a groupby, so it should not be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, count(*) as value from tbl1 a where key < 6 group by a.key) 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: -- One of the subqueries contains a groupby, so it should not be converted to a sort-merge join. +explain +select count(*) from + (select a.key as key, count(*) as value from tbl1 a where key < 6 group by a.key) 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Map 5 (GROUP PARTITION-LEVEL SORT, 1), Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 4 <- Reducer 3 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: key + Group By Operator + aggregations: count() + bucketGroup: true + keys: key (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) + Map 5 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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, count(*) as value from tbl1 a where key < 6 group by a.key) 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, count(*) as value from tbl1 a where key < 6 group by a.key) 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 #### +8 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_11.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_11.q.out new file mode 100644 index 0000000..f519b17 --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_11.q.out @@ -0,0 +1,1460 @@ +PREHOOK: query: -- small 1 part, 2 bucket & big 2 part, 4 bucket + +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: -- small 1 part, 2 bucket & big 2 part, 4 bucket + +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 1 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Map 4 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 0 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Reducer 3 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +38 +PREHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +-- The tables are only bucketed and not sorted, the join should not be converted +-- Currenly, a join is only converted to a sort-merge join without a hint, automatic conversion to +-- bucketized mapjoin is not done +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +-- The tables are only bucketed and not sorted, the join should not be converted +-- Currenly, a join is only converted to a sort-merge join without a hint, automatic conversion to +-- bucketized mapjoin is not done +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 1 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Map 4 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 0 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Reducer 3 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +38 +PREHOOK: query: -- The join is converted to a bucketed mapjoin with a mapjoin hint +explain extended select /*+ mapjoin(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The join is converted to a bucketed mapjoin with a mapjoin hint +explain extended select /*+ mapjoin(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_HINTLIST + TOK_HINT + TOK_MAPJOIN + TOK_HINTARGLIST + a + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 1 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Map 4 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 0 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Reducer 3 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select /*+ mapjoin(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +POSTHOOK: query: select /*+ mapjoin(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +38 +PREHOOK: query: -- HIVE-7023 +explain extended select /* + MAPJOIN(a,b) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key JOIN bucket_big c ON a.key = c.key +PREHOOK: type: QUERY +POSTHOOK: query: -- HIVE-7023 +explain extended select /* + MAPJOIN(a,b) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key JOIN bucket_big c ON a.key = c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_TABREF + TOK_TABNAME + bucket_big + c + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + c + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_HINTLIST + TOK_HINT + TOK_MAPJOIN + TOK_HINTARGLIST + a + b + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 1 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Map 4 + Map Operator Tree: + TableScan + alias: c + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 2 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [c] + /bucket_big/ds=2008-04-09 [c] + Map 5 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 0 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + condition expressions: + 0 + 1 + 2 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Reducer 3 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select /* + MAPJOIN(a,b) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key JOIN bucket_big c ON a.key = c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +POSTHOOK: query: select /* + MAPJOIN(a,b) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key JOIN bucket_big c ON a.key = c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +180 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out new file mode 100644 index 0000000..d24fc1f --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out @@ -0,0 +1,621 @@ +PREHOOK: query: -- small 1 part, 2 bucket & big 2 part, 4 bucket + +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: -- small 1 part, 2 bucket & big 2 part, 4 bucket + +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) +CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: CREATE TABLE bucket_medium (key string, value string) partitioned by (ds string) +CLUSTERED BY (key) SORTED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_medium +POSTHOOK: query: CREATE TABLE bucket_medium (key string, value string) partitioned by (ds string) +CLUSTERED BY (key) SORTED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_medium +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_medium +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_medium +POSTHOOK: Output: default@bucket_medium@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_medium@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_medium@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_medium@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_medium@ds=2008-04-08 +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 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_JOIN + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_medium + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_TABREF + TOK_TABNAME + bucket_big + c + = + . + TOK_TABLE_OR_COL + c + key + . + TOK_TABLE_OR_COL + b + key + TOK_TABREF + TOK_TABNAME + bucket_medium + d + = + . + TOK_TABLE_OR_COL + c + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: d + GatherStats: false + Reduce Output Operator + sort order: + tag: 1 + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 3 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_medium + numFiles 3 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_medium { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 170 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 3 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_medium + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_medium { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_medium + name: default.bucket_medium + Truncated Path -> Alias: + /bucket_medium/ds=2008-04-08 [d] + Map 4 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 1 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 3 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_medium + numFiles 3 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_medium { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 170 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 3 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_medium + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_medium { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_medium + name: default.bucket_medium + Truncated Path -> Alias: + /bucket_medium/ds=2008-04-08 [b] + Map 6 + Map Operator Tree: + TableScan + alias: c + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 2 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [c] + /bucket_big/ds=2008-04-09 [c] + Map 7 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + tag: 0 + auto parallelism: true + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 114 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_small + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_small { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_small + name: default.bucket_small + Truncated Path -> Alias: + /bucket_small/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Reducer 3 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + Reducer 5 + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 + 1 + 2 + Reduce Output Operator + sort order: + tag: 0 + auto parallelism: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +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 +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_medium +PREHOOK: Input: default@bucket_medium@ds=2008-04-08 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +POSTHOOK: 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 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_medium +POSTHOOK: Input: default@bucket_medium@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +570 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_13.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_13.q.out new file mode 100644 index 0000000..c3a2f47 --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_13.q.out @@ -0,0 +1,694 @@ +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 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +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 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +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: CREATE TABLE dest1(k1 int, k2 int) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest1 +POSTHOOK: query: CREATE TABLE dest1(k1 int, k2 int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest1 +PREHOOK: query: CREATE TABLE dest2(k1 string, k2 string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dest2 +POSTHOOK: query: CREATE TABLE dest2(k1 string, k2 string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dest2 +PREHOOK: query: -- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-4 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-4, Stage-5 + Stage-0 depends on stages: Stage-3 + Stage-6 depends on stages: Stage-0 + Stage-1 depends on stages: Stage-3 + Stage-7 depends on stages: Stage-1 + Stage-5 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col5, _col6 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + 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-4 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + Select Operator + expressions: _col0 (type: int), _col2 (type: int) + 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 + name: default.dest1 + + Stage: Stage-3 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + 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 + name: default.dest1 + + Stage: Stage-6 + Stats-Aggr Operator + + Stage: Stage-1 + Move Operator + tables: + replace: true + 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 + name: default.dest2 + + Stage: Stage-7 + Stats-Aggr Operator + + Stage: Stage-5 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + Select Operator + expressions: _col1 (type: string), _col3 (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 + name: default.dest2 + +PREHOOK: query: from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +PREHOOK: Output: default@dest1 +PREHOOK: Output: default@dest2 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +POSTHOOK: Output: default@dest1 +POSTHOOK: Output: default@dest2 +POSTHOOK: Lineage: dest1.k1 SIMPLE [(tbl1)a.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest1.k2 SIMPLE [(tbl2)b.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest2.k1 SIMPLE [(tbl1)a.FieldSchema(name:value, type:string, comment:null), ] +POSTHOOK: Lineage: dest2.k2 SIMPLE [(tbl2)b.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select * from dest1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest1 +#### A masked pattern was here #### +POSTHOOK: query: select * from dest1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest1 +#### A masked pattern was here #### +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +2 2 +4 4 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +8 8 +9 9 +PREHOOK: query: select * from dest2 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest2 +#### A masked pattern was here #### +POSTHOOK: query: select * from dest2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest2 +#### A masked pattern was here #### +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_2 val_2 +val_4 val_4 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_8 val_8 +val_9 val_9 +PREHOOK: query: -- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-4 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-4, Stage-5 + Stage-0 depends on stages: Stage-3 + Stage-6 depends on stages: Stage-0 + Stage-1 depends on stages: Stage-3 + Stage-7 depends on stages: Stage-1 + Stage-5 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col5, _col6 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + 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-4 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + Select Operator + expressions: _col0 (type: int), _col2 (type: int) + 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 + name: default.dest1 + + Stage: Stage-3 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + 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 + name: default.dest1 + + Stage: Stage-6 + Stats-Aggr Operator + + Stage: Stage-1 + Move Operator + tables: + replace: true + 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 + name: default.dest2 + + Stage: Stage-7 + Stats-Aggr Operator + + Stage: Stage-5 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + Select Operator + expressions: _col1 (type: string), _col3 (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 + name: default.dest2 + +PREHOOK: query: from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +PREHOOK: Output: default@dest1 +PREHOOK: Output: default@dest2 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +POSTHOOK: Output: default@dest1 +POSTHOOK: Output: default@dest2 +POSTHOOK: Lineage: dest1.k1 SIMPLE [(tbl1)a.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest1.k2 SIMPLE [(tbl2)b.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest2.k1 SIMPLE [(tbl1)a.FieldSchema(name:value, type:string, comment:null), ] +POSTHOOK: Lineage: dest2.k2 SIMPLE [(tbl2)b.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select * from dest1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest1 +#### A masked pattern was here #### +POSTHOOK: query: select * from dest1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest1 +#### A masked pattern was here #### +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +2 2 +4 4 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +8 8 +9 9 +PREHOOK: query: select * from dest2 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest2 +#### A masked pattern was here #### +POSTHOOK: query: select * from dest2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest2 +#### A masked pattern was here #### +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_2 val_2 +val_4 val_4 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_8 val_8 +val_9 val_9 +PREHOOK: query: -- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join followed by a mutli-insert +explain +from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-4 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-4, Stage-5 + Stage-0 depends on stages: Stage-3 + Stage-6 depends on stages: Stage-0 + Stage-1 depends on stages: Stage-3 + Stage-7 depends on stages: Stage-1 + Stage-5 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col5, _col6 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + 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-4 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + Select Operator + expressions: _col0 (type: int), _col2 (type: int) + 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 + name: default.dest1 + + Stage: Stage-3 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + 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 + name: default.dest1 + + Stage: Stage-6 + Stats-Aggr Operator + + Stage: Stage-1 + Move Operator + tables: + replace: true + 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 + name: default.dest2 + + Stage: Stage-7 + Stats-Aggr Operator + + Stage: Stage-5 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + Select Operator + expressions: _col1 (type: string), _col3 (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 + name: default.dest2 + +PREHOOK: query: from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +PREHOOK: Output: default@dest1 +PREHOOK: Output: default@dest2 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: from ( + SELECT a.key key1, a.value value1, b.key key2, b.value value2 + FROM tbl1 a JOIN tbl2 b + ON a.key = b.key ) subq +INSERT OVERWRITE TABLE dest1 select key1, key2 +INSERT OVERWRITE TABLE dest2 select value1, value2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +POSTHOOK: Output: default@dest1 +POSTHOOK: Output: default@dest2 +POSTHOOK: Lineage: dest1.k1 SIMPLE [(tbl1)a.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest1.k2 SIMPLE [(tbl2)b.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest2.k1 SIMPLE [(tbl1)a.FieldSchema(name:value, type:string, comment:null), ] +POSTHOOK: Lineage: dest2.k2 SIMPLE [(tbl2)b.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select * from dest1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest1 +#### A masked pattern was here #### +POSTHOOK: query: select * from dest1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest1 +#### A masked pattern was here #### +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +2 2 +4 4 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +5 5 +8 8 +9 9 +PREHOOK: query: select * from dest2 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest2 +#### A masked pattern was here #### +POSTHOOK: query: select * from dest2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest2 +#### A masked pattern was here #### +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_0 val_0 +val_2 val_2 +val_4 val_4 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_5 val_5 +val_8 val_8 +val_9 val_9 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_14.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_14.q.out new file mode 100644 index 0000000..71dd51d --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_14.q.out @@ -0,0 +1,195 @@ +PREHOOK: query: 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: 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 < 20 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl1 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table tbl1 select * from src where key < 20 +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 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +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: -- Since tbl1 is the bigger table, tbl1 Left Outer Join tbl2 can be performed +explain +select count(*) FROM tbl1 a LEFT OUTER JOIN tbl2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since tbl1 is the bigger table, tbl1 Left Outer Join tbl2 can be performed +explain +select count(*) FROM tbl1 a LEFT OUTER 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a LEFT OUTER 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 count(*) FROM tbl1 a LEFT OUTER JOIN tbl2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +32 +PREHOOK: query: insert overwrite table tbl2 select * from src where key < 200 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl2 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table tbl2 select * from src where key < 200 +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: -- Since tbl2 is the bigger table, tbl1 Right Outer Join tbl2 can be performed +explain +select count(*) FROM tbl1 a RIGHT OUTER JOIN tbl2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since tbl2 is the bigger table, tbl1 Right Outer Join tbl2 can be performed +explain +select count(*) FROM tbl1 a RIGHT OUTER 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a RIGHT OUTER 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 count(*) FROM tbl1 a RIGHT OUTER JOIN tbl2 b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +207 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_15.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_15.q.out new file mode 100644 index 0000000..6f17195 --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_15.q.out @@ -0,0 +1,158 @@ +PREHOOK: query: 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: 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 < 20 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl1 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table tbl1 select * from src where key < 20 +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 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +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: explain +select count(*) FROM tbl1 a LEFT OUTER JOIN tbl2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select count(*) FROM tbl1 a LEFT OUTER 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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: explain +select count(*) FROM tbl1 a RIGHT OUTER JOIN tbl2 b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select count(*) FROM tbl1 a RIGHT OUTER 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_16.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_16.q.out new file mode 100644 index 0000000..bb53fed --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_16.q.out @@ -0,0 +1,254 @@ +PREHOOK: query: CREATE TABLE stage_bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (file_tag STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@stage_bucket_big +POSTHOOK: query: CREATE TABLE stage_bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (file_tag STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@stage_bucket_big +PREHOOK: query: CREATE TABLE bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (day STRING, pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (day STRING, pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: CREATE TABLE stage_bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (file_tag STRING) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@stage_bucket_small +POSTHOOK: query: CREATE TABLE stage_bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (file_tag STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@stage_bucket_small +PREHOOK: query: CREATE TABLE bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: CREATE TABLE bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='1') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@stage_bucket_small +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='1') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@stage_bucket_small +POSTHOOK: Output: default@stage_bucket_small@file_tag=1 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='2') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@stage_bucket_small +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='2') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@stage_bucket_small +POSTHOOK: Output: default@stage_bucket_small@file_tag=2 +PREHOOK: query: insert overwrite table bucket_small partition(pri) +select +key, +value, +file_tag as pri +from +stage_bucket_small +where file_tag between 1 and 2 +PREHOOK: type: QUERY +PREHOOK: Input: default@stage_bucket_small +PREHOOK: Input: default@stage_bucket_small@file_tag=1 +PREHOOK: Input: default@stage_bucket_small@file_tag=2 +PREHOOK: Output: default@bucket_small +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table bucket_small partition(pri) +select +key, +value, +file_tag as pri +from +stage_bucket_small +where file_tag between 1 and 2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@stage_bucket_small +POSTHOOK: Input: default@stage_bucket_small@file_tag=1 +POSTHOOK: Input: default@stage_bucket_small@file_tag=2 +POSTHOOK: Output: default@bucket_small@pri=1 +POSTHOOK: Output: default@bucket_small@pri=2 +POSTHOOK: Lineage: bucket_small PARTITION(pri=1).key SIMPLE [(stage_bucket_small)stage_bucket_small.FieldSchema(name:key, type:bigint, comment:null), ] +POSTHOOK: Lineage: bucket_small PARTITION(pri=1).value SIMPLE [(stage_bucket_small)stage_bucket_small.FieldSchema(name:value, type:string, comment:null), ] +POSTHOOK: Lineage: bucket_small PARTITION(pri=2).key SIMPLE [(stage_bucket_small)stage_bucket_small.FieldSchema(name:key, type:bigint, comment:null), ] +POSTHOOK: Lineage: bucket_small PARTITION(pri=2).value SIMPLE [(stage_bucket_small)stage_bucket_small.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' overwrite into table stage_bucket_big partition (file_tag='1') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@stage_bucket_big +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' overwrite into table stage_bucket_big partition (file_tag='1') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@stage_bucket_big +POSTHOOK: Output: default@stage_bucket_big@file_tag=1 +PREHOOK: query: insert overwrite table bucket_big partition(day,pri) +select +key, +value, +'day1' as day, +1 as pri +from +stage_bucket_big +where +file_tag='1' +PREHOOK: type: QUERY +PREHOOK: Input: default@stage_bucket_big +PREHOOK: Input: default@stage_bucket_big@file_tag=1 +PREHOOK: Output: default@bucket_big +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table bucket_big partition(day,pri) +select +key, +value, +'day1' as day, +1 as pri +from +stage_bucket_big +where +file_tag='1' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@stage_bucket_big +POSTHOOK: Input: default@stage_bucket_big@file_tag=1 +POSTHOOK: Output: default@bucket_big@day=day1/pri=1 +POSTHOOK: Lineage: bucket_big PARTITION(day=day1,pri=1).key SIMPLE [(stage_bucket_big)stage_bucket_big.FieldSchema(name:key, type:bigint, comment:null), ] +POSTHOOK: Lineage: bucket_big PARTITION(day=day1,pri=1).value SIMPLE [(stage_bucket_big)stage_bucket_big.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: select +a.key , +a.value , +b.value , +'day1' as day, +1 as pri +from +( +select +key, +value +from bucket_big where day='day1' +) a +left outer join +( +select +key, +value +from bucket_small +where pri between 1 and 2 +) b +on +(a.key = b.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@day=day1/pri=1 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@pri=1 +PREHOOK: Input: default@bucket_small@pri=2 +#### A masked pattern was here #### +POSTHOOK: query: select +a.key , +a.value , +b.value , +'day1' as day, +1 as pri +from +( +select +key, +value +from bucket_big where day='day1' +) a +left outer join +( +select +key, +value +from bucket_small +where pri between 1 and 2 +) b +on +(a.key = b.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@day=day1/pri=1 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@pri=1 +POSTHOOK: Input: default@bucket_small@pri=2 +#### A masked pattern was here #### +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +374 val_374 val_374 day1 1 +374 val_374 val_374 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out new file mode 100644 index 0000000..e5ea5a3 --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out @@ -0,0 +1,551 @@ +PREHOOK: query: -- small 1 part, 4 bucket & big 2 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: -- small 1 part, 4 bucket & big 2 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: -- Since the leftmost table is assumed as the big table, arrange the tables in the join accordingly +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since the leftmost table is assumed as the big table, arrange the tables in the join accordingly +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +38 +PREHOOK: query: -- The mapjoin should fail resulting in the sort-merge join +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The mapjoin should fail resulting in the sort-merge join +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +#### A masked pattern was here #### +38 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out new file mode 100644 index 0000000..5eff6a3 --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out @@ -0,0 +1,633 @@ +PREHOOK: query: -- small 2 part, 2 bucket & big 1 part, 4 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: -- small 2 part, 2 bucket & big 1 part, 4 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +38 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +38 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +38 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out new file mode 100644 index 0000000..4743d17 --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out @@ -0,0 +1,649 @@ +PREHOOK: query: -- small 2 part, 4 bucket & big 1 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: -- small 2 part, 4 bucket & big 1 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +38 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +38 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +38 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out new file mode 100644 index 0000000..7d5a3cc --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out @@ -0,0 +1,584 @@ +PREHOOK: query: -- small no part, 4 bucket & big no part, 2 bucket +CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: -- small no part, 4 bucket & big no part, 2 bucket +CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +PREHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: bucket_big + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big [b] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_small +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_small +#### A masked pattern was here #### +19 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: bucket_big + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_small +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_small +#### A masked pattern was here #### +19 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: bucket_big + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_small +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_small +#### A masked pattern was here #### +19 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out new file mode 100644 index 0000000..673919d --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out @@ -0,0 +1,1048 @@ +PREHOOK: query: 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: 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: CREATE TABLE tbl3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl3 +POSTHOOK: query: CREATE TABLE tbl3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl3 +PREHOOK: query: CREATE TABLE tbl4(key int, value string) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@tbl4 +POSTHOOK: query: CREATE TABLE tbl4(key int, value string) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tbl4 +PREHOOK: query: insert overwrite table tbl1 select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl1 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table tbl1 select * from src +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 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl2 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table tbl2 select * from src +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: insert overwrite table tbl3 select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl3 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table tbl3 select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tbl3 +POSTHOOK: Lineage: tbl3.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tbl3.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: insert overwrite table tbl4 select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tbl4 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: insert overwrite table tbl4 select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tbl4 +POSTHOOK: Lineage: tbl4.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tbl4.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- A SMB join is being followed by a regular join on a non-bucketed table on a different key + +-- Three tests below are all the same query with different alias, which changes dispatch order of GenMapRedWalker +-- This is dependent to iteration order of HashMap, so can be meaningless in non-sun jdk +-- b = TS[0]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8]-SEL[9]-FS[10] +-- c = TS[1]-RS[7]-JOIN[8] +-- a = TS[2]-MAPJOIN[11] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join is being followed by a regular join on a non-bucketed table on a different key + +-- Three tests below are all the same query with different alias, which changes dispatch order of GenMapRedWalker +-- This is dependent to iteration order of HashMap, so can be meaningless in non-sun jdk +-- b = TS[0]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8]-SEL[9]-FS[10] +-- c = TS[1]-RS[7]-JOIN[8] +-- a = TS[2]-MAPJOIN[11] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: c + Filter Operator + predicate: value is not null (type: boolean) + Reduce Output Operator + key expressions: value (type: string) + sort order: + + Map-reduce partition columns: value (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (key is not null and value is not null) (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {value} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col1 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- d = TS[0]-RS[7]-JOIN[8]-SEL[9]-FS[10] +-- b = TS[1]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8] +-- a = TS[2]-MAPJOIN[11] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src d on d.value = a.value +PREHOOK: type: QUERY +POSTHOOK: query: -- d = TS[0]-RS[7]-JOIN[8]-SEL[9]-FS[10] +-- b = TS[1]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8] +-- a = TS[2]-MAPJOIN[11] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src d on d.value = a.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: d + Filter Operator + predicate: value is not null (type: boolean) + Reduce Output Operator + key expressions: value (type: string) + sort order: + + Map-reduce partition columns: value (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (key is not null and value is not null) (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {value} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col1 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join src d on d.value = a.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src d on d.value = a.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- b = TS[0]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8]-SEL[9]-FS[10] +-- a = TS[1]-MAPJOIN[11] +-- h = TS[2]-RS[7]-JOIN[8] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src h on h.value = a.value +PREHOOK: type: QUERY +POSTHOOK: query: -- b = TS[0]-OP[13]-MAPJOIN[11]-RS[6]-JOIN[8]-SEL[9]-FS[10] +-- a = TS[1]-MAPJOIN[11] +-- h = TS[2]-RS[7]-JOIN[8] +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src h on h.value = a.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (key is not null and value is not null) (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {value} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col1 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: h + Filter Operator + predicate: value is not null (type: boolean) + Reduce Output Operator + key expressions: value (type: string) + sort order: + + Map-reduce partition columns: value (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join src h on h.value = a.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src h on h.value = a.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- A SMB join is being followed by a regular join on a non-bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join is being followed by a regular join on a non-bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: UDFToDouble(key) is not null (type: boolean) + Reduce Output Operator + key expressions: UDFToDouble(key) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(key) (type: double) + Map 4 + Map Operator Tree: + TableScan + alias: c + Filter Operator + predicate: UDFToDouble(key) is not null (type: boolean) + Reduce Output Operator + key expressions: UDFToDouble(key) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(key) (type: double) + Map 5 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: UDFToDouble(key) is not null (type: boolean) + Reduce Output Operator + key expressions: UDFToDouble(key) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(key) (type: double) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + condition expressions: + 0 + 1 + 2 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- A SMB join is being followed by a regular join on a bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join is being followed by a regular join on a bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + condition expressions: + 0 + 1 + 2 + keys: + 0 key (type: int) + 1 key (type: int) + 2 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +PREHOOK: Input: default@tbl3 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +POSTHOOK: Input: default@tbl3 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- A SMB join is being followed by a regular join on a bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join is being followed by a regular join on a bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: c + Filter Operator + predicate: value is not null (type: boolean) + Reduce Output Operator + key expressions: value (type: string) + sort order: + + Map-reduce partition columns: value (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (key is not null and value is not null) (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {value} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col1 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +PREHOOK: Input: default@tbl4 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +POSTHOOK: Input: default@tbl4 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- A SMB join is being followed by a regular join on a non-bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join is being followed by a regular join on a non-bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: c + Filter Operator + predicate: value is not null (type: boolean) + Reduce Output Operator + key expressions: value (type: string) + sort order: + + Map-reduce partition columns: value (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (key is not null and value is not null) (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {value} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col1 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.value = a.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- A SMB join is being followed by a regular join on a non-bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join is being followed by a regular join on a non-bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: UDFToDouble(key) is not null (type: boolean) + Reduce Output Operator + key expressions: UDFToDouble(key) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(key) (type: double) + Map 4 + Map Operator Tree: + TableScan + alias: c + Filter Operator + predicate: UDFToDouble(key) is not null (type: boolean) + Reduce Output Operator + key expressions: UDFToDouble(key) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(key) (type: double) + Map 5 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: UDFToDouble(key) is not null (type: boolean) + Reduce Output Operator + key expressions: UDFToDouble(key) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(key) (type: double) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + condition expressions: + 0 + 1 + 2 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join src c on c.key = a.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- A SMB join is being followed by a regular join on a bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join is being followed by a regular join on a bucketed table on the same key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + condition expressions: + 0 + 1 + 2 + keys: + 0 key (type: int) + 1 key (type: int) + 2 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +PREHOOK: Input: default@tbl3 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl3 c on c.key = a.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +POSTHOOK: Input: default@tbl3 +#### A masked pattern was here #### +2654 +PREHOOK: query: -- A SMB join is being followed by a regular join on a bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value +PREHOOK: type: QUERY +POSTHOOK: query: -- A SMB join is being followed by a regular join on a bucketed table on a different key +explain select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: c + Filter Operator + predicate: value is not null (type: boolean) + Reduce Output Operator + key expressions: value (type: string) + sort order: + + Map-reduce partition columns: value (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (key is not null and value is not null) (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {value} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col1 + Reduce Output Operator + key expressions: _col1 (type: string) + sort order: + + Map-reduce partition columns: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +PREHOOK: Input: default@tbl4 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM tbl1 a JOIN tbl2 b ON a.key = b.key join tbl4 c on c.value = a.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +POSTHOOK: Input: default@tbl4 +#### A masked pattern was here #### +2654 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out new file mode 100644 index 0000000..a78e843 --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out @@ -0,0 +1,819 @@ +PREHOOK: query: -- small 2 part, 4 bucket & big 2 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: -- small 2 part, 4 bucket & big 2 part, 2 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +76 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +76 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 2 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 2750 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +76 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out new file mode 100644 index 0000000..962a98c --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out @@ -0,0 +1,821 @@ +PREHOOK: query: -- small 2 part, 2 bucket & big 2 part, 4 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_small +POSTHOOK: query: -- small 2 part, 2 bucket & big 2 part, 4 bucket +CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_small +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_small@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_small@ds=2008-04-09 +PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@bucket_big +POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@bucket_big +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@bucket_big@ds=2008-04-09 +POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@bucket_big@ds=2008-04-09 +PREHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter +explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_small + a + TOK_TABREF + TOK_TABNAME + bucket_big + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: b + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [b] + /bucket_big/ds=2008-04-09 [b] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +76 +PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +76 +PREHOOK: query: -- The mapjoin should fail resulting in the sort-merge join +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The mapjoin should fail resulting in the sort-merge join +explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + bucket_big + a + TOK_TABREF + TOK_TABNAME + bucket_small + b + = + . + TOK_TABLE_OR_COL + a + key + . + TOK_TABLE_OR_COL + b + key + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_FUNCTIONSTAR + count + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: string) + 1 key (type: string) + Position of Big Table: 0 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + tag: -1 + value expressions: _col0 (type: bigint) + auto parallelism: false + Path -> Alias: +#### A masked pattern was here #### + Path -> Partition: +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-08 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-08 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big +#### A masked pattern was here #### + Partition + base file name: ds=2008-04-09 + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + partition values: + ds 2008-04-09 + properties: + COLUMN_STATS_ACCURATE true + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + numFiles 4 + numRows 0 + partition_columns ds + partition_columns.types string + rawDataSize 0 + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 4 + bucket_field_name key + columns key,value + columns.comments + columns.types string:string +#### A masked pattern was here #### + name default.bucket_big + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket_big { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket_big + name: default.bucket_big + Truncated Path -> Alias: + /bucket_big/ds=2008-04-08 [a] + /bucket_big/ds=2008-04-09 [a] + Reducer 2 + Needs Tagging: false + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0 + columns.types bigint + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@bucket_big +PREHOOK: Input: default@bucket_big@ds=2008-04-08 +PREHOOK: Input: default@bucket_big@ds=2008-04-09 +PREHOOK: Input: default@bucket_small +PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@bucket_big +POSTHOOK: Input: default@bucket_big@ds=2008-04-08 +POSTHOOK: Input: default@bucket_big@ds=2008-04-09 +POSTHOOK: Input: default@bucket_small +POSTHOOK: Input: default@bucket_small@ds=2008-04-08 +POSTHOOK: Input: default@bucket_small@ds=2008-04-09 +#### A masked pattern was here #### +76 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out new file mode 100644 index 0000000..b96b2b2 --- /dev/null +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out @@ -0,0 +1,2917 @@ +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 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +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 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +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: -- 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 sub-query. It should be converted to a sort-merge join +explain +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 +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 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _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) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: 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 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +0 9 +2 1 +4 1 +5 9 +8 1 +9 1 +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-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _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) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _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) + Map 4 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _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) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + 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) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {KEY.reducesinkkey0} {VALUE._col0} + 1 {VALUE._col0} + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col3 (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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + 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) + + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 8) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 mapside +-- 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 mapside +-- 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-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Select Operator + expressions: (key + 1) (type: int) + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: a + Select Operator + expressions: (key + 1) (type: int) + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 +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: -- The left table is a sub-query and the right table 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: -- The left table is a sub-query and the right table 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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: -- The right table is a sub-query and the left table is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The right table is a sub-query and the left table is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: int) + 1 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.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-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + condition expressions: + 0 + 1 + 2 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + 2 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 sub-query. It should be converted to a sort-merge join +explain +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 +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 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 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _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) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: 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 +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: 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 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tbl1 +POSTHOOK: Input: default@tbl2 +#### A masked pattern was here #### +0 9 +2 1 +4 1 +5 9 +8 1 +9 1 +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-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _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) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1), Reducer 5 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 5 <- Map 4 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _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) + Map 4 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0 + Select Operator + expressions: _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) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + 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) + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {KEY.reducesinkkey0} {VALUE._col0} + 1 {VALUE._col0} + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint), _col3 (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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int), _col1 (type: bigint) + 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) + + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 8) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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: -- The left table is a sub-query and the right table 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: -- The left table is a sub-query and the right table 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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: -- The right table is a sub-query and the left table is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key +PREHOOK: type: QUERY +POSTHOOK: query: -- The right table is a sub-query and the left table is not. +-- It should be converted to a sort-merge join. +explain +select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 key (type: int) + 1 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.key +PREHOOK: type: QUERY +PREHOOK: Input: default@tbl1 +PREHOOK: Input: default@tbl2 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from tbl1 a + join + (select a.key as key, a.value as value from tbl2 a where key < 6) subq1 + on a.key = subq1.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-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: ((key < 6) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + condition expressions: + 0 + 1 + 2 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + 2 _col0 (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int) + outputColumnNames: _col0 + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 + 1 + keys: + 0 _col0 (type: int) + 1 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out new file mode 100644 index 0000000..7901cfb --- /dev/null +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out @@ -0,0 +1,789 @@ +PREHOOK: query: -- Create bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table1 +POSTHOOK: query: -- Create bucketed and sorted tables +CREATE TABLE test_table1 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table1 +PREHOOK: query: CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table2 +POSTHOOK: query: CREATE TABLE test_table2 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table2 +PREHOOK: query: CREATE TABLE test_table3 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table3 +POSTHOOK: query: CREATE TABLE test_table3 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table3 +PREHOOK: query: CREATE TABLE test_table4 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table4 +POSTHOOK: query: CREATE TABLE test_table4 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table4 +PREHOOK: query: CREATE TABLE test_table5 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table5 +POSTHOOK: query: CREATE TABLE test_table5 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table5 +PREHOOK: query: CREATE TABLE test_table6 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table6 +POSTHOOK: query: CREATE TABLE test_table6 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table6 +PREHOOK: query: CREATE TABLE test_table7 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table7 +POSTHOOK: query: CREATE TABLE test_table7 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table7 +PREHOOK: query: CREATE TABLE test_table8 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@test_table8 +POSTHOOK: query: CREATE TABLE test_table8 (key INT, value STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test_table8 +PREHOOK: query: INSERT OVERWRITE TABLE test_table1 +SELECT * FROM src WHERE key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table1 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: INSERT OVERWRITE TABLE test_table1 +SELECT * FROM src WHERE key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table1 +POSTHOOK: Lineage: test_table1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE test_table2 +SELECT * FROM src WHERE key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table2 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: INSERT OVERWRITE TABLE test_table2 +SELECT * FROM src WHERE key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table2 +POSTHOOK: Lineage: test_table2.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE test_table3 +SELECT * FROM src WHERE key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table3 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: INSERT OVERWRITE TABLE test_table3 +SELECT * FROM src WHERE key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table3 +POSTHOOK: Lineage: test_table3.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table3.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE test_table4 +SELECT * FROM src WHERE key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table4 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: INSERT OVERWRITE TABLE test_table4 +SELECT * FROM src WHERE key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table4 +POSTHOOK: Lineage: test_table4.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table4.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE test_table5 +SELECT * FROM src WHERE key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table5 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: INSERT OVERWRITE TABLE test_table5 +SELECT * FROM src WHERE key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table5 +POSTHOOK: Lineage: test_table5.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table5.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE test_table6 +SELECT * FROM src WHERE key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table6 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: INSERT OVERWRITE TABLE test_table6 +SELECT * FROM src WHERE key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table6 +POSTHOOK: Lineage: test_table6.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table6.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE test_table7 +SELECT * FROM src WHERE key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table7 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: INSERT OVERWRITE TABLE test_table7 +SELECT * FROM src WHERE key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table7 +POSTHOOK: Lineage: test_table7.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table7.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE test_table8 +SELECT * FROM src WHERE key < 10 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test_table8 +[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to +POSTHOOK: query: INSERT OVERWRITE TABLE test_table8 +SELECT * FROM src WHERE key < 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test_table8 +POSTHOOK: Lineage: test_table8.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test_table8.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- Mapjoin followed by a aggregation should be performed in a single MR job upto 7 tables +EXPLAIN +SELECT /*+ mapjoin(b, c, d, e, f, g) */ count(*) +FROM test_table1 a JOIN test_table2 b ON a.key = b.key +JOIN test_table3 c ON a.key = c.key +JOIN test_table4 d ON a.key = d.key +JOIN test_table5 e ON a.key = e.key +JOIN test_table6 f ON a.key = f.key +JOIN test_table7 g ON a.key = g.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Mapjoin followed by a aggregation should be performed in a single MR job upto 7 tables +EXPLAIN +SELECT /*+ mapjoin(b, c, d, e, f, g) */ count(*) +FROM test_table1 a JOIN test_table2 b ON a.key = b.key +JOIN test_table3 c ON a.key = c.key +JOIN test_table4 d ON a.key = d.key +JOIN test_table5 e ON a.key = e.key +JOIN test_table6 f ON a.key = f.key +JOIN test_table7 g ON a.key = g.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1), Map 7 (GROUP PARTITION-LEVEL SORT, 1), Map 8 (GROUP PARTITION-LEVEL SORT, 1), Map 9 (GROUP PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: f + Filter Operator + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: g + Filter Operator + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: d + Filter Operator + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: e + Filter Operator + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 7 + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 8 + Map Operator Tree: + TableScan + alias: c + Filter Operator + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 9 + Map Operator Tree: + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 0 to 2 + Inner Join 0 to 3 + Inner Join 0 to 4 + Inner Join 0 to 5 + Inner Join 0 to 6 + condition expressions: + 0 + 1 + 2 + 3 + 4 + 5 + 6 + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 /*+ mapjoin(b, c, d, e, f, g) */ count(*) +FROM test_table1 a JOIN test_table2 b ON a.key = b.key +JOIN test_table3 c ON a.key = c.key +JOIN test_table4 d ON a.key = d.key +JOIN test_table5 e ON a.key = e.key +JOIN test_table6 f ON a.key = f.key +JOIN test_table7 g ON a.key = g.key +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table1 +PREHOOK: Input: default@test_table2 +PREHOOK: Input: default@test_table3 +PREHOOK: Input: default@test_table4 +PREHOOK: Input: default@test_table5 +PREHOOK: Input: default@test_table6 +PREHOOK: Input: default@test_table7 +#### A masked pattern was here #### +POSTHOOK: query: SELECT /*+ mapjoin(b, c, d, e, f, g) */ count(*) +FROM test_table1 a JOIN test_table2 b ON a.key = b.key +JOIN test_table3 c ON a.key = c.key +JOIN test_table4 d ON a.key = d.key +JOIN test_table5 e ON a.key = e.key +JOIN test_table6 f ON a.key = f.key +JOIN test_table7 g ON a.key = g.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table1 +POSTHOOK: Input: default@test_table2 +POSTHOOK: Input: default@test_table3 +POSTHOOK: Input: default@test_table4 +POSTHOOK: Input: default@test_table5 +POSTHOOK: Input: default@test_table6 +POSTHOOK: Input: default@test_table7 +#### A masked pattern was here #### +4378 +PREHOOK: query: -- It should be automatically converted to a sort-merge join followed by a groupby in +-- a single MR job +EXPLAIN +SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +PREHOOK: type: QUERY +POSTHOOK: query: -- It should be automatically converted to a sort-merge join followed by a groupby in +-- a single MR job +EXPLAIN +SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join0 to 2 + Left Outer Join0 to 3 + Left Outer Join0 to 4 + Left Outer Join0 to 5 + Left Outer Join0 to 6 + condition expressions: + 0 + 1 + 2 + 3 + 4 + 5 + 6 + keys: + 0 key (type: int) + 1 key (type: int) + 2 key (type: int) + 3 key (type: int) + 4 key (type: int) + 5 key (type: int) + 6 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table1 +PREHOOK: Input: default@test_table2 +PREHOOK: Input: default@test_table3 +PREHOOK: Input: default@test_table4 +PREHOOK: Input: default@test_table5 +PREHOOK: Input: default@test_table6 +PREHOOK: Input: default@test_table7 +#### A masked pattern was here #### +POSTHOOK: query: SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table1 +POSTHOOK: Input: default@test_table2 +POSTHOOK: Input: default@test_table3 +POSTHOOK: Input: default@test_table4 +POSTHOOK: Input: default@test_table5 +POSTHOOK: Input: default@test_table6 +POSTHOOK: Input: default@test_table7 +#### A masked pattern was here #### +4378 +PREHOOK: query: EXPLAIN +SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join0 to 2 + Left Outer Join0 to 3 + Left Outer Join0 to 4 + Left Outer Join0 to 5 + Left Outer Join0 to 6 + Left Outer Join0 to 7 + condition expressions: + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + keys: + 0 key (type: int) + 1 key (type: int) + 2 key (type: int) + 3 key (type: int) + 4 key (type: int) + 5 key (type: int) + 6 key (type: int) + 7 key (type: int) + Select Operator + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: bigint) + 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 test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key +PREHOOK: type: QUERY +PREHOOK: Input: default@test_table1 +PREHOOK: Input: default@test_table2 +PREHOOK: Input: default@test_table3 +PREHOOK: Input: default@test_table4 +PREHOOK: Input: default@test_table5 +PREHOOK: Input: default@test_table6 +PREHOOK: Input: default@test_table7 +PREHOOK: Input: default@test_table8 +#### A masked pattern was here #### +POSTHOOK: query: SELECT count(*) +FROM test_table1 a LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test_table1 +POSTHOOK: Input: default@test_table2 +POSTHOOK: Input: default@test_table3 +POSTHOOK: Input: default@test_table4 +POSTHOOK: Input: default@test_table5 +POSTHOOK: Input: default@test_table6 +POSTHOOK: Input: default@test_table7 +POSTHOOK: Input: default@test_table8 +#### A masked pattern was here #### +13126 +PREHOOK: query: -- outer join with max 16 aliases +EXPLAIN +SELECT a.* +FROM test_table1 a +LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key +LEFT OUTER JOIN test_table4 i ON a.key = i.key +LEFT OUTER JOIN test_table5 j ON a.key = j.key +LEFT OUTER JOIN test_table6 k ON a.key = k.key +LEFT OUTER JOIN test_table7 l ON a.key = l.key +LEFT OUTER JOIN test_table8 m ON a.key = m.key +LEFT OUTER JOIN test_table7 n ON a.key = n.key +LEFT OUTER JOIN test_table8 o ON a.key = o.key +LEFT OUTER JOIN test_table4 p ON a.key = p.key +LEFT OUTER JOIN test_table5 q ON a.key = q.key +LEFT OUTER JOIN test_table6 r ON a.key = r.key +LEFT OUTER JOIN test_table7 s ON a.key = s.key +LEFT OUTER JOIN test_table8 t ON a.key = t.key +PREHOOK: type: QUERY +POSTHOOK: query: -- outer join with max 16 aliases +EXPLAIN +SELECT a.* +FROM test_table1 a +LEFT OUTER JOIN test_table2 b ON a.key = b.key +LEFT OUTER JOIN test_table3 c ON a.key = c.key +LEFT OUTER JOIN test_table4 d ON a.key = d.key +LEFT OUTER JOIN test_table5 e ON a.key = e.key +LEFT OUTER JOIN test_table6 f ON a.key = f.key +LEFT OUTER JOIN test_table7 g ON a.key = g.key +LEFT OUTER JOIN test_table8 h ON a.key = h.key +LEFT OUTER JOIN test_table4 i ON a.key = i.key +LEFT OUTER JOIN test_table5 j ON a.key = j.key +LEFT OUTER JOIN test_table6 k ON a.key = k.key +LEFT OUTER JOIN test_table7 l ON a.key = l.key +LEFT OUTER JOIN test_table8 m ON a.key = m.key +LEFT OUTER JOIN test_table7 n ON a.key = n.key +LEFT OUTER JOIN test_table8 o ON a.key = o.key +LEFT OUTER JOIN test_table4 p ON a.key = p.key +LEFT OUTER JOIN test_table5 q ON a.key = q.key +LEFT OUTER JOIN test_table6 r ON a.key = r.key +LEFT OUTER JOIN test_table7 s ON a.key = s.key +LEFT OUTER JOIN test_table8 t ON a.key = t.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP PARTITION-LEVEL SORT, 1), Map 3 (GROUP PARTITION-LEVEL SORT, 1), Map 4 (GROUP PARTITION-LEVEL SORT, 1), Map 5 (GROUP PARTITION-LEVEL SORT, 1), Map 6 (GROUP PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join0 to 2 + Left Outer Join0 to 3 + Left Outer Join0 to 4 + Left Outer Join0 to 5 + Left Outer Join0 to 6 + Left Outer Join0 to 7 + Left Outer Join0 to 8 + Left Outer Join0 to 9 + Left Outer Join0 to 10 + Left Outer Join0 to 11 + Left Outer Join0 to 12 + Left Outer Join0 to 13 + Left Outer Join0 to 14 + Left Outer Join0 to 15 + condition expressions: + 0 {key} {value} + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12 + 13 + 14 + 15 + keys: + 0 key (type: int) + 1 key (type: int) + 2 key (type: int) + 3 key (type: int) + 4 key (type: int) + 5 key (type: int) + 6 key (type: int) + 7 key (type: int) + 8 key (type: int) + 9 key (type: int) + 10 key (type: int) + 11 key (type: int) + 12 key (type: int) + 13 key (type: int) + 14 key (type: int) + 15 key (type: int) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: t + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 4 + Map Operator Tree: + TableScan + alias: s + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 5 + Map Operator Tree: + TableScan + alias: r + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Map 6 + Map Operator Tree: + TableScan + alias: q + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join0 to 2 + Left Outer Join0 to 3 + Left Outer Join0 to 4 + condition expressions: + 0 {KEY.reducesinkkey0} {VALUE._col0} + 1 + 2 + 3 + 4 + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: int), _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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink +