Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java =================================================================== --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (revision 916550) +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (working copy) @@ -200,6 +200,7 @@ HIVEOPTPPD("hive.optimize.ppd", true), // predicate pushdown HIVEOPTGROUPBY("hive.optimize.groupby", true), // optimize group by HIVEOPTBUCKETMAPJOIN("hive.optimize.bucketmapjoin", false), // optimize bucket map join + HIVEOPTSORTMERGEBUCKETMAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false), // try to use sorted merge bucket map join ; public final String varname; Index: data/files/smbbucket_1.txt =================================================================== --- data/files/smbbucket_1.txt (revision 0) +++ data/files/smbbucket_1.txt (revision 0) @@ -0,0 +1,5 @@ +1val_1 +3val_3 +4val_4 +5val_5 +10val_10 Index: data/files/smbbucket_2.txt =================================================================== --- data/files/smbbucket_2.txt (revision 0) +++ data/files/smbbucket_2.txt (revision 0) @@ -0,0 +1,4 @@ +20val_20 +23val_23 +25val_25 +30val_30 Index: data/files/smbbucket_3.txt =================================================================== --- data/files/smbbucket_3.txt (revision 0) +++ data/files/smbbucket_3.txt (revision 0) @@ -0,0 +1,6 @@ +4val_4 +10val_10 +17val_17 +19val_19 +20val_20 +23val_23 Index: ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java (revision 0) @@ -0,0 +1,154 @@ +/** + * 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.exec; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.HashMap; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.MapJoinOperator.MapJoinObjectCtx; +import org.apache.hadoop.hive.ql.exec.persistence.HashMapWrapper; +import org.apache.hadoop.hive.ql.exec.persistence.MapJoinObjectKey; +import org.apache.hadoop.hive.ql.exec.persistence.MapJoinObjectValue; +import org.apache.hadoop.hive.ql.exec.persistence.RowContainer; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption; +import org.apache.hadoop.util.ReflectionUtils; + + +public abstract class AbstractMapJoinOperator extends CommonJoinOperator implements + Serializable { + private static final long serialVersionUID = 1L; + + /** + * The expressions for join inputs's join keys. + */ + protected transient Map> joinKeys; + /** + * The ObjectInspectors for the join inputs's join keys. + */ + protected transient Map> joinKeysObjectInspectors; + /** + * The standard ObjectInspectors for the join inputs's join keys. + */ + protected transient Map> joinKeysStandardObjectInspectors; + + protected transient int posBigTable = -1; // one of the tables that is not in memory + transient int mapJoinRowsKey; // rows for a given key + + protected transient RowContainer> emptyList = null; + + transient int numMapRowsRead; + + private static final transient String[] FATAL_ERR_MSG = { + null, // counter value 0 means no error + "Mapside join size exceeds hive.mapjoin.maxsize. " + + "Please increase that or remove the mapjoin hint." + }; + + transient boolean firstRow; + transient int heartbeatInterval; + + public AbstractMapJoinOperator() { + } + + public AbstractMapJoinOperator(AbstractMapJoinOperator mjop) { + super((CommonJoinOperator)mjop); + } + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + super.initializeOp(hconf); + + numMapRowsRead = 0; + firstRow = true; + heartbeatInterval = HiveConf.getIntVar(hconf, + HiveConf.ConfVars.HIVESENDHEARTBEAT); + + joinKeys = new HashMap>(); + + populateJoinKeyValue(joinKeys, conf.getKeys()); + joinKeysObjectInspectors = getObjectInspectorsFromEvaluators(joinKeys, + inputObjInspectors); + joinKeysStandardObjectInspectors = getStandardObjectInspectors(joinKeysObjectInspectors); + + // all other tables are small, and are cached in the hash table + posBigTable = conf.getPosBigTable(); + + emptyList = new RowContainer>(1, hconf); + RowContainer bigPosRC = getRowContainer(hconf, (byte) posBigTable, + order[posBigTable], joinCacheSize); + storage.put((byte) posBigTable, bigPosRC); + + mapJoinRowsKey = HiveConf.getIntVar(hconf, + HiveConf.ConfVars.HIVEMAPJOINROWSIZE); + + List structFields = ((StructObjectInspector) outputObjInspector) + .getAllStructFieldRefs(); + if (conf.getOutputColumnNames().size() < structFields.size()) { + List structFieldObjectInspectors = new ArrayList(); + for (Byte alias : order) { + int sz = conf.getExprs().get(alias).size(); + List retained = conf.getRetainList().get(alias); + for (int i = 0; i < sz; i++) { + int pos = retained.get(i); + structFieldObjectInspectors.add(structFields.get(pos) + .getFieldObjectInspector()); + } + } + outputObjInspector = ObjectInspectorFactory + .getStandardStructObjectInspector(conf.getOutputColumnNames(), + structFieldObjectInspectors); + } + initializeChildren(hconf); + } + + @Override + protected void fatalErrorMessage(StringBuilder errMsg, long counterCode) { + errMsg.append("Operator " + getOperatorId() + " (id=" + id + "): " + + FATAL_ERR_MSG[(int) counterCode]); + } + + protected void reportProgress() { + // Send some status periodically + numMapRowsRead++; + if (((numMapRowsRead % heartbeatInterval) == 0) && (reporter != null)) { + reporter.progress(); + } + } + + @Override + public int getType() { + return OperatorType.MAPJOIN; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java (working copy) @@ -110,7 +110,7 @@ // be output protected transient JoinCondDesc[] condn; protected transient boolean noOuterJoin; - private transient Object[] dummyObj; // for outer joins, contains the + protected transient Object[] dummyObj; // for outer joins, contains the // potential nulls for the concerned // aliases protected transient RowContainer>[] dummyObjVectors; // empty @@ -140,6 +140,48 @@ transient boolean handleSkewJoin = false; + public CommonJoinOperator() { + } + + public CommonJoinOperator(CommonJoinOperator clone) { + this.joinEmitInterval = clone.joinEmitInterval; + this.joinCacheSize = clone.joinCacheSize; + this.nextSz = clone.nextSz; + this.childOperators = clone.childOperators; + this.parentOperators = clone.parentOperators; + this.counterNames = clone.counterNames; + this.counterNameToEnum = clone.counterNameToEnum; + this.done = clone.done; + this.operatorId = clone.operatorId; + this.storage = clone.storage; + this.condn = clone.condn; + + this.setSchema(clone.getSchema()); + + this.alias = clone.alias; + this.beginTime = clone.beginTime; + this.inputRows = clone.inputRows; + this.childOperatorsArray = clone.childOperatorsArray; + this.childOperatorsTag = clone.childOperatorsTag; + this.colExprMap = clone.colExprMap; + this.counters = clone.counters; + this.dummyObj = clone.dummyObj; + this.dummyObjVectors = clone.dummyObjVectors; + this.forwardCache = clone.forwardCache; + this.groupKeyObject = clone.groupKeyObject; + this.handleSkewJoin = clone.handleSkewJoin; + this.hconf = clone.hconf; + this.id = clone.id; + this.inputObjInspectors = clone.inputObjInspectors; + this.inputRows = clone.inputRows; + this.noOuterJoin = clone.noOuterJoin; + this.numAliases = clone.numAliases; + this.operatorId = clone.operatorId; + this.posToAliasMap = clone.posToAliasMap; + this.spillTableDesc = clone.spillTableDesc; + this.statsMap = clone.statsMap; + } + protected int populateJoinKeyValue(Map> outMap, Map> inputMap) { @@ -224,8 +266,6 @@ protected void initializeOp(Configuration hconf) throws HiveException { this.handleSkewJoin = conf.getHandleSkewJoin(); this.hconf = hconf; - LOG.info("COMMONJOIN " - + ((StructObjectInspector) inputObjInspectors[0]).getTypeName()); totalSz = 0; // Map that contains the rows for each alias storage = new HashMap>>(); @@ -699,7 +739,7 @@ * maintained (inputNulls) where each entry denotes whether the element is to * be used or not (whether it is null or not). The size of the bitvector is * same as the number of inputs under consideration currently. When all inputs - * are accounted for, the output is forwared appropriately. + * are accounted for, the output is forwarded appropriately. */ private void genObject(ArrayList inputNulls, int aliasNum, IntermediateObject intObj, boolean firstRow) throws HiveException { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java (working copy) @@ -68,6 +68,32 @@ private long nextCntr = 1; private String lastInputFile = null; private MapredLocalWork localWork = null; + + private ExecMapperContext execContext = new ExecMapperContext(); + + public static class ExecMapperContext { + boolean inputFileChanged = false; + String currentInputFile; + JobConf jc; + public boolean isInputFileChanged() { + return inputFileChanged; + } + public void setInputFileChanged(boolean inputFileChanged) { + this.inputFileChanged = inputFileChanged; + } + public String getCurrentInputFile() { + return currentInputFile; + } + public void setCurrentInputFile(String currentInputFile) { + this.currentInputFile = currentInputFile; + } + public JobConf getJc() { + return jc; + } + public void setJc(JobConf jc) { + this.jc = jc; + } + } @Override public void configure(JobConf job) { @@ -86,6 +112,7 @@ } try { jc = job; + execContext.jc = jc; // create map and fetch operators MapredWork mrwork = Utilities.getMapRedWork(job); mo = new MapOperator(); @@ -93,8 +120,10 @@ // initialize map operator mo.setChildren(job); l4j.info(mo.dump(0)); + mo.setExecContext(execContext); + mo.initializeLocalWork(jc); mo.initialize(jc, null); - + // initialize map local work localWork = mrwork.getMapLocalWork(); if (localWork == null) { @@ -112,6 +141,7 @@ for (Map.Entry entry : fetchOperators.entrySet()) { Operator forwardOp = localWork.getAliasToWork() .get(entry.getKey()); + forwardOp.setExecContext(execContext); // All the operators need to be initialized before process forwardOp.initialize(jc, new ObjectInspector[] {entry.getValue() .getOutputObjectInspector()}); @@ -141,11 +171,12 @@ mo.setReporter(rp); } - if (localWork != null - && (this.lastInputFile == null || - (localWork.getInputFileChangeSensitive() && inputFileChanged()))) { + if(inputFileChanged()) { + if (this.localWork != null + && (localWork.getInputFileChangeSensitive() || this.lastInputFile == null)) { + processMapLocalWork(localWork.getInputFileChangeSensitive()); + } this.lastInputFile = HiveConf.getVar(jc, HiveConf.ConfVars.HADOOPMAPFILENAME); - processMapLocalWork(localWork.getInputFileChangeSensitive()); } try { @@ -188,10 +219,13 @@ */ private boolean inputFileChanged() { String currentInputFile = HiveConf.getVar(jc, HiveConf.ConfVars.HADOOPMAPFILENAME); + execContext.currentInputFile = currentInputFile; if (this.lastInputFile == null || !this.lastInputFile.equals(currentInputFile)) { + execContext.inputFileChanged = true; return true; } + execContext.inputFileChanged = false; return false; } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java (working copy) @@ -18,7 +18,6 @@ package org.apache.hadoop.hive.ql.exec; -import java.io.File; import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; @@ -50,32 +49,14 @@ /** * Map side Join operator implementation. */ -public class MapJoinOperator extends CommonJoinOperator implements +public class MapJoinOperator extends AbstractMapJoinOperator implements Serializable { private static final long serialVersionUID = 1L; private static final Log LOG = LogFactory.getLog(MapJoinOperator.class .getName()); - /** - * The expressions for join inputs's join keys. - */ - protected transient Map> joinKeys; - /** - * The ObjectInspectors for the join inputs's join keys. - */ - protected transient Map> joinKeysObjectInspectors; - /** - * The standard ObjectInspectors for the join inputs's join keys. - */ - protected transient Map> joinKeysStandardObjectInspectors; - - private transient int posBigTable; // one of the tables that is not in memory - transient int mapJoinRowsKey; // rows for a given key - protected transient Map> mapJoinTables; - protected transient RowContainer> emptyList = null; - private static final transient String[] FATAL_ERR_MSG = { null, // counter value 0 means no error "Mapside join size exceeds hive.mapjoin.maxsize. " @@ -135,43 +116,30 @@ return mapMetadata; } - transient boolean firstRow; - transient int metadataKeyTag; transient int[] metadataValueTag; - transient List hTables; - transient int numMapRowsRead; - transient int heartbeatInterval; transient int maxMapJoinSize; + + public MapJoinOperator() { + } + + public MapJoinOperator(AbstractMapJoinOperator mjop) { + super(mjop); + } @Override protected void initializeOp(Configuration hconf) throws HiveException { super.initializeOp(hconf); - numMapRowsRead = 0; - - firstRow = true; - heartbeatInterval = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVESENDHEARTBEAT); + maxMapJoinSize = HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVEMAXMAPJOINSIZE); - joinKeys = new HashMap>(); - - populateJoinKeyValue(joinKeys, conf.getKeys()); - joinKeysObjectInspectors = getObjectInspectorsFromEvaluators(joinKeys, - inputObjInspectors); - joinKeysStandardObjectInspectors = getStandardObjectInspectors(joinKeysObjectInspectors); - - // all other tables are small, and are cached in the hash table - posBigTable = conf.getPosBigTable(); - metadataValueTag = new int[numAliases]; for (int pos = 0; pos < numAliases; pos++) { metadataValueTag[pos] = -1; } mapJoinTables = new HashMap>(); - hTables = new ArrayList(); // initialize the hash tables for other tables for (int pos = 0; pos < numAliases; pos++) { @@ -186,33 +154,6 @@ mapJoinTables.put(Byte.valueOf((byte) pos), hashTable); } - - emptyList = new RowContainer>(1, hconf); - RowContainer bigPosRC = getRowContainer(hconf, (byte) posBigTable, - order[posBigTable], joinCacheSize); - storage.put((byte) posBigTable, bigPosRC); - - mapJoinRowsKey = HiveConf.getIntVar(hconf, - HiveConf.ConfVars.HIVEMAPJOINROWSIZE); - - List structFields = ((StructObjectInspector) outputObjInspector) - .getAllStructFieldRefs(); - if (conf.getOutputColumnNames().size() < structFields.size()) { - List structFieldObjectInspectors = new ArrayList(); - for (Byte alias : order) { - int sz = conf.getExprs().get(alias).size(); - List retained = conf.getRetainList().get(alias); - for (int i = 0; i < sz; i++) { - int pos = retained.get(i); - structFieldObjectInspectors.add(structFields.get(pos) - .getFieldObjectInspector()); - } - } - outputObjInspector = ObjectInspectorFactory - .getStandardStructObjectInspector(conf.getOutputColumnNames(), - structFieldObjectInspectors); - } - initializeChildren(hconf); } @Override @@ -258,11 +199,7 @@ firstRow = false; } - // Send some status periodically - numMapRowsRead++; - if (((numMapRowsRead % heartbeatInterval) == 0) && (reporter != null)) { - reporter.progress(); - } + reportProgress(); if ((numMapRowsRead > maxMapJoinSize) && (reporter != null) && (counterNameToEnum != null)) { @@ -380,7 +317,7 @@ } super.closeOp(abort); } - + /** * Implements the getName function for the Node Interface. * Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java (working copy) @@ -29,6 +29,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.ExecMapper.ExecMapperContext; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.Explain; @@ -67,6 +68,8 @@ * run-time while extracting the operator specific counts. */ protected HashMap counterNameToEnum; + + private transient ExecMapperContext execContext; private static int seqId; @@ -284,7 +287,7 @@ } return true; } - + /** * Initializes operators only if all parents have been initialized. Calls * operator specific initializer which then initializes child ops. @@ -340,11 +343,23 @@ } // derived classes can set this to different object if needed outputObjInspector = inputObjInspectors[0]; - + + //pass the exec context to child operators + passExecContext(this.execContext); + initializeOp(hconf); LOG.info("Initialization Done " + id + " " + getName()); } - + + public void initializeLocalWork(Configuration hconf) throws HiveException { + if (childOperators != null) { + for (int i =0; i childOp = this.childOperators.get(i); + childOp.initializeLocalWork(hconf); + } + } + } + /** * Operator specific initialization. */ @@ -371,6 +386,18 @@ } } } + + /** + * Pass the execContext reference to every child operator + */ + public void passExecContext(ExecMapperContext execContext) { + this.setExecContext(execContext); + if(childOperators != null) { + for (int i = 0; i < childOperators.size(); i++) { + childOperators.get(i).passExecContext(execContext); + } + } + } /** * Collects all the parent's output object inspectors and calls actual @@ -398,7 +425,7 @@ // call the actual operator initialization function initialize(hconf, null); } - + /** * Process the row. * @@ -470,7 +497,7 @@ LOG.debug("End group Done"); } - private boolean allInitializedParentsAreClosed() { + protected boolean allInitializedParentsAreClosed() { if (parentOperators != null) { for (Operator parent : parentOperators) { if (!(parent.state == State.CLOSE || parent.state == State.UNINIT)) { @@ -1117,4 +1144,18 @@ public Object getGroupKeyObject() { return groupKeyObject; } + + public ExecMapperContext getExecContext() { + return execContext; + } + + public void setExecContext(ExecMapperContext execContext) { + this.execContext = execContext; + if(this.childOperators != null) { + for (int i = 0; i op = this.childOperators.get(i); + op.setExecContext(execContext); + } + } + } } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java (working copy) @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.plan.LimitDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; import org.apache.hadoop.hive.ql.plan.ScriptDesc; import org.apache.hadoop.hive.ql.plan.SelectDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; @@ -74,6 +75,7 @@ opvec.add(new OpTuple(GroupByDesc.class, GroupByOperator.class)); opvec.add(new OpTuple(JoinDesc.class, JoinOperator.class)); opvec.add(new OpTuple(MapJoinDesc.class, MapJoinOperator.class)); + opvec.add(new OpTuple(SMBJoinDesc.class, SMBMapJoinOperator.class)); opvec.add(new OpTuple(LimitDesc.class, LimitOperator.class)); opvec.add(new OpTuple(TableScanDesc.class, TableScanOperator.class)); opvec.add(new OpTuple(UnionDesc.class, UnionOperator.class)); Index: ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java (revision 0) @@ -0,0 +1,512 @@ +/** + * 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.exec; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.persistence.RowContainer; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.FetchWork; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; +import org.apache.hadoop.hive.ql.plan.MapredLocalWork; +import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; +import org.apache.hadoop.hive.ql.plan.MapredLocalWork.BucketMapJoinContext; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.WritableComparator; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * Sorted Merge Map Join Operator. + */ +public class SMBMapJoinOperator extends AbstractMapJoinOperator implements + Serializable { + + private static final long serialVersionUID = 1L; + + private static final Log LOG = LogFactory.getLog(SMBMapJoinOperator.class + .getName()); + + private MapredLocalWork localWork = null; + private Map fetchOperators; + transient Map> keyWritables; + transient Map> nextKeyWritables; + HashMap>> nextGroupStorage; + HashMap>> candidateStorage; + + transient HashMap tagToAlias; + private transient HashMap fetchOpDone = new HashMap(); + private transient HashMap foundNextKeyGroup = new HashMap(); + transient boolean firstFetchHappened = false; + transient boolean localWorkInited = false; + + public SMBMapJoinOperator() { + } + + public SMBMapJoinOperator(AbstractMapJoinOperator mapJoinOp) { + super(mapJoinOp); + } + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + super.initializeOp(hconf); + + firstRow = true; + + nextGroupStorage = new HashMap>>(); + candidateStorage = new HashMap>>(); + int bucketSize = HiveConf.getIntVar(hconf, + HiveConf.ConfVars.HIVEMAPJOINBUCKETCACHESIZE); + byte storePos = (byte) 0; + for (Byte alias : order) { + RowContainer rc = getRowContainer(hconf, storePos, alias, bucketSize); + nextGroupStorage.put((byte) storePos, rc); + RowContainer candidateRC = getRowContainer(hconf, storePos, alias, + bucketSize); + candidateStorage.put(alias, candidateRC); + storePos++; + } + tagToAlias = conf.getTagToAlias(); + keyWritables = new HashMap>(); + nextKeyWritables = new HashMap>(); + + for (Byte alias : order) { + if(alias != (byte) posBigTable) { + fetchOpDone.put(alias, Boolean.FALSE);; + } + foundNextKeyGroup.put(alias, Boolean.FALSE); + } + } + + @Override + public void initializeLocalWork(Configuration hconf) throws HiveException { + initializeMapredLocalWork(this.getConf(), hconf, this.getConf().getLocalWork(), LOG); + super.initializeLocalWork(hconf); + } + + public void initializeMapredLocalWork(MapJoinDesc conf, Configuration hconf, + MapredLocalWork localWork, Log l4j) throws HiveException { + if (localWork == null || localWorkInited) { + return; + } + localWorkInited = true; + this.localWork = localWork; + fetchOperators = new HashMap(); + // create map local operators + for (Map.Entry entry : localWork.getAliasToFetchWork() + .entrySet()) { + fetchOperators.put(entry.getKey(), new FetchOperator(entry.getValue(), + new JobConf(hconf))); + if (l4j != null) { + l4j.info("fetchoperator for " + entry.getKey() + " created"); + } + } + + for (Map.Entry entry : fetchOperators.entrySet()) { + Operator forwardOp = localWork.getAliasToWork() + .get(entry.getKey()); + // All the operators need to be initialized before process + forwardOp.setExecContext(this.getExecContext()); + forwardOp.initialize(this.getExecContext().jc, new ObjectInspector[] {entry.getValue() + .getOutputObjectInspector()}); + l4j.info("fetchoperator for " + entry.getKey() + " initialized"); + } + } + + @Override + public void processOp(Object row, int tag) throws HiveException { + + if (this.getExecContext().inputFileChanged) { + if(firstFetchHappened) { + //we need to first join and flush out data left by the previous file. + joinFinalLeftData(); + } + //set up the fetch operator for the new input file. + for (Map.Entry entry : fetchOperators.entrySet()) { + String alias = entry.getKey(); + FetchOperator fetchOp = entry.getValue(); + fetchOp.clearFetchContext(); + setUpFetchOpContext(fetchOp, alias); + } + this.getExecContext().inputFileChanged = false; + firstFetchHappened = false; + } + + if (!firstFetchHappened) { + firstFetchHappened = true; + // fetch the first group for all small table aliases + for (Byte t : order) { + if(t != (byte)posBigTable) { + fetchNextGroup(t); + } + } + } + + byte alias = (byte) tag; + // compute keys and values as StandardObjects + ArrayList key = computeValues(row, joinKeys.get(alias), + joinKeysObjectInspectors.get(alias)); + ArrayList value = computeValues(row, joinValues.get(alias), + joinValuesObjectInspectors.get(alias)); + + //have we reached a new key group? + boolean nextKeyGroup = processKey(alias, key); + if (nextKeyGroup) { + //assert this.nextGroupStorage.get(alias).size() == 0; + this.nextGroupStorage.get(alias).add(value); + foundNextKeyGroup.put((byte) tag, Boolean.TRUE); + if (tag != posBigTable) { + return; + } + } + + reportProgress(); + + // the big table has reached a new key group. try to let the small tables + // catch up with the big table. + if (nextKeyGroup) { + assert tag == (byte)posBigTable; + List smallestPos = null; + do { + smallestPos = joinOneGroup(); + //jump out the loop if we need input from the big table + } while (smallestPos != null && smallestPos.size() > 0 + && !smallestPos.contains((byte)this.posBigTable)); + + return; + } + + assert !nextKeyGroup; + candidateStorage.get((byte) tag).add(value); + } + + /* + * this happens either when the input file of the big table is changed or in + * closeop. It needs to fetch all the left data from the small tables and try + * to join them. + */ + private void joinFinalLeftData() throws HiveException { + RowContainer bigTblRowContainer = this.candidateStorage.get((byte)this.posBigTable); + + boolean allFetchOpDone = allFetchOpDone(); + // if all left data in small tables are less than and equal to the left data + // in big table, let's them catch up + while (bigTblRowContainer != null && bigTblRowContainer.size() > 0 + && !allFetchOpDone) { + joinOneGroup(); + bigTblRowContainer = this.candidateStorage.get((byte)this.posBigTable); + allFetchOpDone = allFetchOpDone(); + } + + if (allFetchOpDone + && this.candidateStorage.get((byte) this.posBigTable).size() > 0) { + // if all fetch operator for small tables are done and there are data left + // in big table + joinOneGroup(); + } else { + while (!allFetchOpDone) { + List ret = joinOneGroup(); + if (ret == null || ret.size() == 0) { + break; + } + + reportProgress(); + + allFetchOpDone = allFetchOpDone(); + } + //one final table left + for (byte t : order) { + if(this.foundNextKeyGroup.get(t) && this.nextKeyWritables.get(t) != null) { + promoteNextGroupToCandidate(t); + } + } + joinOneGroup(); + } + } + + private boolean allFetchOpDone() { + boolean allFetchOpDone = true; + for (Byte tag : order) { + if(tag == (byte) posBigTable) { + continue; + } + allFetchOpDone = allFetchOpDone && fetchOpDone.get(tag); + } + return allFetchOpDone; + } + + private List joinOneGroup() throws HiveException { + int smallestPos = -1; + smallestPos = findMostSmallKey(); + List listOfNeedFetchNext = null; + if(smallestPos >= 0) { + listOfNeedFetchNext = joinObject(smallestPos); + if (listOfNeedFetchNext.size() > 0) { + // listOfNeedFetchNext contains all tables that we have joined data in their + // candidateStorage, and we need to clear candidate storage and promote their + // nextGroupStorage to candidateStorage and fetch data until we reach a + // new group. + for (Byte b : listOfNeedFetchNext) { + fetchNextGroup(b); + } + } + } + return listOfNeedFetchNext; + } + + private List joinObject(int smallestPos) throws HiveException { + List needFetchList = new ArrayList(); + ArrayList smallKey = keyWritables.get((byte) smallestPos); + needFetchList.add((byte)smallestPos); + this.storage.put((byte) smallestPos, this.candidateStorage.get((byte) smallestPos)); + for (Byte i : order) { + if ((byte) smallestPos == i) { + continue; + } + ArrayList key = keyWritables.get(i); + if (key == null) { + putDummyOrEmpty(i); + } else { + int cmp = compareKeys(key, smallKey); + if (cmp == 0) { + this.storage.put((byte) i, this.candidateStorage + .get((byte) i)); + needFetchList.add(i); + continue; + } else { + putDummyOrEmpty(i); + } + } + } + checkAndGenObject(); + for (Byte pos : needFetchList) { + this.candidateStorage.get(pos).clear(); + this.keyWritables.remove(pos); + } + return needFetchList; + } + + private void fetchNextGroup(Byte t) throws HiveException { + if (foundNextKeyGroup.get(t)) { + // first promote the next group to be the current group if we reached a + // new group in the previous fetch + if (this.nextKeyWritables.get(t) != null) { + promoteNextGroupToCandidate(t); + } else { + this.keyWritables.remove(t); + this.candidateStorage.remove(t); + this.nextGroupStorage.remove(t); + } + foundNextKeyGroup.put(t, Boolean.FALSE); + } + //for the big table, we only need to promote the next group to the current group. + if(t == (byte)posBigTable) { + return; + } + + //for tables other than the big table, we need to fetch more data until reach a new group or done. + while (!foundNextKeyGroup.get(t)) { + if (fetchOpDone.get(t)) { + break; + } + fetchOneRow(t); + } + if (!foundNextKeyGroup.get(t) && fetchOpDone.get(t)) { + this.nextKeyWritables.remove(t); + } + } + + private void promoteNextGroupToCandidate(Byte t) throws HiveException { + this.keyWritables.put(t, this.nextKeyWritables.get(t)); + this.nextKeyWritables.remove(t); + RowContainer> oldRowContainer = this.candidateStorage.get(t); + oldRowContainer.clear(); + this.candidateStorage.put(t, this.nextGroupStorage.get(t)); + this.nextGroupStorage.put(t, oldRowContainer); + } + + private int compareKeys (ArrayList k1, ArrayList k2) { + int ret = 0; + for (int i = 0; i < k1.size() && i < k1.size(); i++) { + WritableComparable key_1 = (WritableComparable) k1.get(i); + WritableComparable key_2 = (WritableComparable) k2.get(i); + ret = WritableComparator.get(key_1.getClass()).compare(key_1, key_2); + if(ret != 0) { + return ret; + } + } + return k1.size() - k2.size(); + } + + private void putDummyOrEmpty(Byte i) { + // put a empty list or null + if (noOuterJoin) { + storage.put(i, emptyList); + } else { + storage.put(i, dummyObjVectors[i.intValue()]); + } + } + + private int findMostSmallKey() { + byte index = -1; + ArrayList mostSmallOne = null; + + for (byte i : order) { + ArrayList key = keyWritables.get(i); + if (key == null) { + continue; + } + if (mostSmallOne == null) { + mostSmallOne = key; + index = i; + continue; + } + int cmp = compareKeys(key, mostSmallOne); + if (cmp < 0) { + mostSmallOne = key; + index = i; + continue; + } + } + return index; + } + + private boolean processKey(byte alias, ArrayList key) + throws HiveException { + ArrayList keyWritable = keyWritables.get(alias); + if (keyWritable == null) { + //the first group. + keyWritables.put(alias, key); + return false; + } else { + int cmp = compareKeys(key, keyWritable);; + if (cmp != 0) { + nextKeyWritables.put(alias, key); + return true; + } + return false; + } + } + + private void setUpFetchOpContext(FetchOperator fetchOp, String alias) { + String currentInputFile = this.getExecContext().currentInputFile; + BucketMapJoinContext bucketMatcherCxt = this.localWork + .getBucketMapjoinContext(); + Class bucketMatcherCls = bucketMatcherCxt + .getBucketMatcherClass(); + BucketMatcher bucketMatcher = (BucketMatcher) ReflectionUtils.newInstance( + bucketMatcherCls, null); + bucketMatcher.setAliasBucketFileNameMapping(bucketMatcherCxt + .getAliasBucketFileNameMapping()); + List aliasFiles = bucketMatcher.getAliasBucketFiles(currentInputFile, + bucketMatcherCxt.getMapJoinBigTableAlias(), alias); + Iterator iter = aliasFiles.iterator(); + fetchOp.setupContext(iter, null); + } + + private void fetchOneRow(byte tag) { + if (fetchOperators != null) { + String tble = this.tagToAlias.get(tag); + FetchOperator fetchOp = fetchOperators.get(tble); + + Operator forwardOp = localWork.getAliasToWork() + .get(tble); + try { + InspectableObject row = fetchOp.getNextRow(); + if (row == null) { + this.fetchOpDone.put(tag, Boolean.TRUE); + return; + } + forwardOp.process(row.o, 0); + // check if any operator had a fatal error or early exit during + // execution + if (forwardOp.getDone()) { + this.fetchOpDone.put(tag, Boolean.TRUE); + } + } catch (Throwable e) { + if (e instanceof OutOfMemoryError) { + // Don't create a new object if we are already out of memory + throw (OutOfMemoryError) e; + } else { + throw new RuntimeException("Map local work failed", e); + } + } + } + } + + transient boolean closeCalled = false; + @Override + public void closeOp(boolean abort) throws HiveException { + if(closeCalled) { + return; + } + closeCalled = true; + joinFinalLeftData(); + this.firstFetchHappened = false; + //clean up + for (Byte alias : order) { + if(alias != (byte) posBigTable) { + fetchOpDone.put(alias, Boolean.FALSE);; + } + foundNextKeyGroup.put(alias, Boolean.FALSE); + } + + localWorkInited = false; + + super.closeOp(abort); + if (fetchOperators != null) { + for (Map.Entry entry : fetchOperators.entrySet()) { + Operator forwardOp = localWork + .getAliasToWork().get(entry.getKey()); + forwardOp.close(abort); + } + } + } + + protected boolean allInitializedParentsAreClosed() { + return true; + } + + /** + * Implements the getName function for the Node Interface. + * + * @return the name of the operator + */ + @Override + public String getName() { + return "MAPJOIN"; + } + + @Override + public int getType() { + return OperatorType.MAPJOIN; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketMapJoinOptimizer.java (working copy) @@ -192,6 +192,8 @@ for (int index = 0; index < joinAliases.size(); index++) { String alias = joinAliases.get(index); TableScanOperator tso = (TableScanOperator) topOps.get(alias); + if (tso == null) + return null; Table tbl = topToTable.get(tso); if(tbl.isPartitioned()) { PrunedPartitionList prunedParts = null; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java (working copy) @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.exec.ConditionalTask; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; @@ -54,6 +55,7 @@ import org.apache.hadoop.hive.ql.plan.ExtractDesc; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.LoadFileDesc; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; @@ -342,7 +344,7 @@ return dest; } - MapJoinOperator currMapJoinOp = ctx.getCurrMapJoinOp(); + AbstractMapJoinOperator currMapJoinOp = ctx.getCurrMapJoinOp(); if (currMapJoinOp != null) { opTaskMap.put(null, currTask); Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java (working copy) @@ -26,6 +26,7 @@ import java.util.Set; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; @@ -35,6 +36,7 @@ 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.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; /** @@ -140,7 +142,7 @@ String taskTmpDir; TableDesc tt_desc; Operator rootMapJoinOp; - MapJoinOperator oldMapJoin; + AbstractMapJoinOperator oldMapJoin; public GenMRMapJoinCtx() { taskTmpDir = null; @@ -157,7 +159,7 @@ */ public GenMRMapJoinCtx(String taskTmpDir, TableDesc tt_desc, Operator rootMapJoinOp, - MapJoinOperator oldMapJoin) { + AbstractMapJoinOperator oldMapJoin) { this.taskTmpDir = taskTmpDir; this.tt_desc = tt_desc; this.rootMapJoinOp = rootMapJoinOp; @@ -198,7 +200,7 @@ /** * @return the oldMapJoin */ - public MapJoinOperator getOldMapJoin() { + public AbstractMapJoinOperator getOldMapJoin() { return oldMapJoin; } @@ -206,7 +208,7 @@ * @param oldMapJoin * the oldMapJoin to set */ - public void setOldMapJoin(MapJoinOperator oldMapJoin) { + public void setOldMapJoin(AbstractMapJoinOperator oldMapJoin) { this.oldMapJoin = oldMapJoin; } } @@ -214,7 +216,7 @@ private HiveConf conf; private HashMap, Task> opTaskMap; private HashMap unionTaskMap; - private HashMap mapJoinTaskMap; + private HashMap, GenMRMapJoinCtx> mapJoinTaskMap; private List> seenOps; private List seenFileSinkOps; @@ -226,7 +228,7 @@ private Task currTask; private Operator currTopOp; private UnionOperator currUnionOp; - private MapJoinOperator currMapJoinOp; + private AbstractMapJoinOperator currMapJoinOp; private String currAliasId; private List> rootOps; @@ -289,7 +291,7 @@ rootOps = new ArrayList>(); rootOps.addAll(parseCtx.getTopOps().values()); unionTaskMap = new HashMap(); - mapJoinTaskMap = new HashMap(); + mapJoinTaskMap = new HashMap, GenMRMapJoinCtx>(); } /** @@ -456,7 +458,7 @@ this.currUnionOp = currUnionOp; } - public MapJoinOperator getCurrMapJoinOp() { + public AbstractMapJoinOperator getCurrMapJoinOp() { return currMapJoinOp; } @@ -464,7 +466,7 @@ * @param currMapJoinOp * current map join operator */ - public void setCurrMapJoinOp(MapJoinOperator currMapJoinOp) { + public void setCurrMapJoinOp(AbstractMapJoinOperator currMapJoinOp) { this.currMapJoinOp = currMapJoinOp; } @@ -491,11 +493,11 @@ unionTaskMap.put(op, uTask); } - public GenMRMapJoinCtx getMapJoinCtx(MapJoinOperator op) { + public GenMRMapJoinCtx getMapJoinCtx(AbstractMapJoinOperator op) { return mapJoinTaskMap.get(op); } - public void setMapJoinCtx(MapJoinOperator op, GenMRMapJoinCtx mjCtx) { + public void setMapJoinCtx(AbstractMapJoinOperator op, GenMRMapJoinCtx mjCtx) { mapJoinTaskMap.put(op, mjCtx); } Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java (working copy) @@ -27,6 +27,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.OperatorFactory; @@ -45,6 +46,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; import org.apache.hadoop.hive.ql.plan.PlanUtils; @@ -141,7 +143,7 @@ // If there is a mapjoin at position 'pos' if (uPrsCtx.getMapJoinSubq(pos)) { - MapJoinOperator mjOp = ctx.getCurrMapJoinOp(); + AbstractMapJoinOperator mjOp = ctx.getCurrMapJoinOp(); assert mjOp != null; GenMRMapJoinCtx mjCtx = ctx.getMapJoinCtx(mjOp); assert mjCtx != null; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (working copy) @@ -33,11 +33,13 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.OperatorFactory; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.exec.UnionOperator; @@ -157,7 +159,7 @@ // The mapjoin has already been encountered. Some context must be stored // about that if (readInputMapJoin) { - MapJoinOperator currMapJoinOp = opProcCtx.getCurrMapJoinOp(); + AbstractMapJoinOperator currMapJoinOp = (AbstractMapJoinOperator) opProcCtx.getCurrMapJoinOp(); assert currMapJoinOp != null; boolean local = ((pos == -1) || (pos == (currMapJoinOp.getConf()) .getPosBigTable())) ? false : true; @@ -217,7 +219,7 @@ seenOps.add(currTopOp); boolean local = (pos == desc.getPosBigTable()) ? false : true; setTaskPlan(currAliasId, currTopOp, plan, local, opProcCtx); - setupBucketMapJoinInfo(plan, (MapJoinOperator)op); + setupBucketMapJoinInfo(plan, (AbstractMapJoinOperator)op); } opProcCtx.setCurrTask(currTask); @@ -226,16 +228,35 @@ } private static void setupBucketMapJoinInfo(MapredWork plan, - MapJoinOperator currMapJoinOp) { + AbstractMapJoinOperator currMapJoinOp) { if (currMapJoinOp != null) { LinkedHashMap>> aliasBucketFileNameMapping = currMapJoinOp.getConf().getAliasBucketFileNameMapping(); if(aliasBucketFileNameMapping!= null) { MapredLocalWork localPlan = plan.getMapLocalWork(); - if (localPlan == null) { - localPlan = new MapredLocalWork( - new LinkedHashMap>(), - new LinkedHashMap()); + if(localPlan == null) { + if(currMapJoinOp instanceof SMBMapJoinOperator) { + localPlan = ((SMBMapJoinOperator)currMapJoinOp).getConf().getLocalWork(); + } + if (localPlan == null) { + localPlan = new MapredLocalWork( + new LinkedHashMap>(), + new LinkedHashMap()); + } + } 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(currMapJoinOp instanceof SMBMapJoinOperator) { + plan.setMapLocalWork(null); + ((SMBMapJoinOperator)currMapJoinOp).getConf().setLocalWork(localPlan); + } else { plan.setMapLocalWork(localPlan); } BucketMapJoinContext bucketMJCxt = new BucketMapJoinContext(); @@ -364,11 +385,14 @@ : true; } setTaskPlan(currAliasId, currTopOp, plan, local, opProcCtx); + if(op instanceof AbstractMapJoinOperator) { + setupBucketMapJoinInfo(plan, (AbstractMapJoinOperator)op); + } } currTopOp = null; opProcCtx.setCurrTopOp(currTopOp); } else if (opProcCtx.getCurrMapJoinOp() != null) { - MapJoinOperator mjOp = opProcCtx.getCurrMapJoinOp(); + AbstractMapJoinOperator mjOp = (AbstractMapJoinOperator) opProcCtx.getCurrMapJoinOp(); if (readUnionData) { initUnionPlan(opProcCtx, currTask, false); } else { @@ -376,7 +400,7 @@ // In case of map-join followed by map-join, the file needs to be // obtained from the old map join - MapJoinOperator oldMapJoin = mjCtx.getOldMapJoin(); + AbstractMapJoinOperator oldMapJoin = (AbstractMapJoinOperator) mjCtx.getOldMapJoin(); String taskTmpDir = null; TableDesc tt_desc = null; Operator rootOp = null; @@ -819,8 +843,8 @@ setTaskPlan(taskTmpDir, streamDesc, ts_op, cplan, local, tt_desc); // This can be cleaned up as a function table in future - if (op instanceof MapJoinOperator) { - MapJoinOperator mjOp = (MapJoinOperator) op; + if (op instanceof AbstractMapJoinOperator) { + AbstractMapJoinOperator mjOp = (AbstractMapJoinOperator) op; opProcCtx.setCurrMapJoinOp(mjOp); GenMRMapJoinCtx mjCtx = opProcCtx.getMapJoinCtx(mjOp); if (mjCtx == null) { Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinFactory.java (working copy) @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.OperatorFactory; @@ -43,6 +44,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.plan.TableDesc; @@ -52,7 +54,7 @@ */ public final class MapJoinFactory { - public static int getPositionParent(MapJoinOperator op, Stack stack) { + public static int getPositionParent(AbstractMapJoinOperator op, Stack stack) { int pos = 0; int size = stack.size(); assert size >= 2 && stack.get(size - 1) == op; @@ -72,7 +74,7 @@ @Override public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { - MapJoinOperator mapJoin = (MapJoinOperator) nd; + AbstractMapJoinOperator mapJoin = (AbstractMapJoinOperator) nd; GenMRProcContext ctx = (GenMRProcContext) procCtx; // find the branch on which this processor was invoked @@ -122,7 +124,7 @@ @Override public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { - MapJoinOperator mapJoin = (MapJoinOperator) nd; + AbstractMapJoinOperator mapJoin = (AbstractMapJoinOperator) nd; GenMRProcContext opProcCtx = (GenMRProcContext) procCtx; MapredWork cplan = GenMapRedUtils.getMapRedWork(); @@ -133,7 +135,7 @@ // find the branch on which this processor was invoked int pos = getPositionParent(mapJoin, stack); - boolean local = (pos == (mapJoin.getConf()).getPosBigTable()) ? false + boolean local = (pos == ((MapJoinDesc)(mapJoin.getConf())).getPosBigTable()) ? false : true; GenMapRedUtils.splitTasks(mapJoin, currTask, redTask, opProcCtx, false, @@ -180,7 +182,7 @@ Object... nodeOutputs) throws SemanticException { SelectOperator sel = (SelectOperator) nd; - MapJoinOperator mapJoin = (MapJoinOperator) sel.getParentOperators().get( + AbstractMapJoinOperator mapJoin = (AbstractMapJoinOperator) sel.getParentOperators().get( 0); assert sel.getParentOperators().size() == 1; @@ -188,7 +190,7 @@ ParseContext parseCtx = ctx.getParseCtx(); // is the mapjoin followed by a reducer - List listMapJoinOps = parseCtx + List> listMapJoinOps = parseCtx .getListMapJoinOpsNoReducer(); if (listMapJoinOps.contains(mapJoin)) { @@ -263,11 +265,11 @@ @Override public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { - MapJoinOperator mapJoin = (MapJoinOperator) nd; + AbstractMapJoinOperator mapJoin = (AbstractMapJoinOperator) nd; GenMRProcContext ctx = (GenMRProcContext) procCtx; ctx.getParseCtx(); - MapJoinOperator oldMapJoin = ctx.getCurrMapJoinOp(); + AbstractMapJoinOperator oldMapJoin = ctx.getCurrMapJoinOp(); assert oldMapJoin != null; GenMRMapJoinCtx mjCtx = ctx.getMapJoinCtx(mapJoin); if (mjCtx != null) { @@ -335,7 +337,7 @@ UnionOperator currUnion = ctx.getCurrUnionOp(); assert currUnion != null; ctx.getUnionTask(currUnion); - MapJoinOperator mapJoin = (MapJoinOperator) nd; + AbstractMapJoinOperator mapJoin = (AbstractMapJoinOperator) nd; // find the branch on which this processor was invoked int pos = getPositionParent(mapJoin, stack); Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java (working copy) @@ -28,7 +28,13 @@ import java.util.Set; import java.util.Stack; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; @@ -36,6 +42,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; import org.apache.hadoop.hive.ql.exec.RowSchema; import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; import org.apache.hadoop.hive.ql.lib.Dispatcher; import org.apache.hadoop.hive.ql.lib.GraphWalker; @@ -44,21 +51,29 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.lib.Rule; import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner; +import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; import org.apache.hadoop.hive.ql.parse.ErrorMsg; import org.apache.hadoop.hive.ql.parse.GenMapRedWalker; import org.apache.hadoop.hive.ql.parse.OpParseContext; import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; import org.apache.hadoop.hive.ql.parse.QBJoinTree; import org.apache.hadoop.hive.ql.parse.RowResolver; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; import org.apache.hadoop.hive.ql.plan.JoinDesc; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.PlanUtils; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; import org.apache.hadoop.hive.ql.plan.SelectDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; /** * Implementation of one of the rule-based map join optimization. User passes @@ -68,6 +83,9 @@ * implemented, this transformation can also be done based on costs. */ public class MapJoinProcessor implements Transform { + + private static final Log LOG = LogFactory.getLog(MapJoinProcessor.class.getName()); + private ParseContext pGraphContext; /** @@ -84,7 +102,7 @@ pGraphContext.getOpParseCtx().put(op, ctx); return op; } - + /** * convert a regular join to a a map-side join. * @@ -101,18 +119,12 @@ // outer join cannot be performed on a table which is being cached JoinDesc desc = op.getConf(); org.apache.hadoop.hive.ql.plan.JoinCondDesc[] condns = desc.getConds(); - for (org.apache.hadoop.hive.ql.plan.JoinCondDesc condn : condns) { - if (condn.getType() == JoinDesc.FULL_OUTER_JOIN) { - throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg()); - } - if ((condn.getType() == JoinDesc.LEFT_OUTER_JOIN) - && (condn.getLeft() != mapJoinPos)) { - throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg()); - } - if ((condn.getType() == JoinDesc.RIGHT_OUTER_JOIN) - && (condn.getRight() != mapJoinPos)) { - throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg()); - } + HiveConf hiveConf = pGraphContext.getConf(); + boolean noCheckOuterJoin = HiveConf.getBoolVar(hiveConf, + HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN) + && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN); + if (!noCheckOuterJoin) { + checkMapJoin(mapJoinPos, condns); } RowResolver oldOutputRS = pctx.getOpParseCtx().get(op).getRR(); @@ -243,7 +255,7 @@ keyTableDesc, valueExprMap, valueTableDescs, outputColumnNames, mapJoinPos, joinCondns), new RowSchema(outputRS.getColumnInfos()), newPar), outputRS); - + mapJoinOp.getConf().setReversedExprs(op.getConf().getReversedExprs()); mapJoinOp.setColumnExprMap(colExprMap); @@ -264,6 +276,24 @@ return mapJoinOp; } + public static void checkMapJoin(int mapJoinPos, + org.apache.hadoop.hive.ql.plan.JoinCondDesc[] condns) + throws SemanticException { + for (org.apache.hadoop.hive.ql.plan.JoinCondDesc condn : condns) { + if (condn.getType() == JoinDesc.FULL_OUTER_JOIN) { + throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg()); + } + if ((condn.getType() == JoinDesc.LEFT_OUTER_JOIN) + && (condn.getLeft() != mapJoinPos)) { + throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg()); + } + if ((condn.getType() == JoinDesc.RIGHT_OUTER_JOIN) + && (condn.getRight() != mapJoinPos)) { + throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg()); + } + } + } + private void genSelectPlan(ParseContext pctx, MapJoinOperator input) throws SemanticException { List> childOps = input.getChildOperators(); @@ -396,7 +426,7 @@ } // Go over the list and find if a reducer is not needed - List listMapJoinOpsNoRed = new ArrayList(); + List> listMapJoinOpsNoRed = new ArrayList>(); // create a walker which walks the tree in a DFS manner while maintaining // the operator stack. @@ -461,8 +491,8 @@ Object... nodeOutputs) throws SemanticException { MapJoinWalkerCtx ctx = (MapJoinWalkerCtx) procCtx; - MapJoinOperator mapJoin = ctx.getCurrMapJoinOp(); - List listRejectedMapJoins = ctx + AbstractMapJoinOperator mapJoin = ctx.getCurrMapJoinOp(); + List> listRejectedMapJoins = ctx .getListRejectedMapJoins(); // the mapjoin has already been handled @@ -471,9 +501,9 @@ return null; } - List listMapJoinsNoRed = ctx.getListMapJoinsNoRed(); + List> listMapJoinsNoRed = ctx.getListMapJoinsNoRed(); if (listMapJoinsNoRed == null) { - listMapJoinsNoRed = new ArrayList(); + listMapJoinsNoRed = new ArrayList>(); } listMapJoinsNoRed.add(mapJoin); ctx.setListMapJoins(listMapJoinsNoRed); @@ -494,11 +524,11 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { MapJoinWalkerCtx ctx = (MapJoinWalkerCtx) procCtx; - MapJoinOperator mapJoin = ctx.getCurrMapJoinOp(); - List listRejectedMapJoins = ctx + AbstractMapJoinOperator mapJoin = ctx.getCurrMapJoinOp(); + List> listRejectedMapJoins = ctx .getListRejectedMapJoins(); if (listRejectedMapJoins == null) { - listRejectedMapJoins = new ArrayList(); + listRejectedMapJoins = new ArrayList>(); } listRejectedMapJoins.add(mapJoin); ctx.setListRejectedMapJoins(listRejectedMapJoins); @@ -543,23 +573,23 @@ * */ public static class MapJoinWalkerCtx implements NodeProcessorCtx { - private List listMapJoinsNoRed; - private List listRejectedMapJoins; - private MapJoinOperator currMapJoinOp; + private List> listMapJoinsNoRed; + private List> listRejectedMapJoins; + private AbstractMapJoinOperator currMapJoinOp; /** * @param listMapJoinsNoRed */ - public MapJoinWalkerCtx(List listMapJoinsNoRed) { + public MapJoinWalkerCtx(List> listMapJoinsNoRed) { this.listMapJoinsNoRed = listMapJoinsNoRed; currMapJoinOp = null; - listRejectedMapJoins = new ArrayList(); + listRejectedMapJoins = new ArrayList>(); } /** * @return the listMapJoins */ - public List getListMapJoinsNoRed() { + public List> getListMapJoinsNoRed() { return listMapJoinsNoRed; } @@ -567,14 +597,14 @@ * @param listMapJoinsNoRed * the listMapJoins to set */ - public void setListMapJoins(List listMapJoinsNoRed) { + public void setListMapJoins(List> listMapJoinsNoRed) { this.listMapJoinsNoRed = listMapJoinsNoRed; } /** * @return the currMapJoinOp */ - public MapJoinOperator getCurrMapJoinOp() { + public AbstractMapJoinOperator getCurrMapJoinOp() { return currMapJoinOp; } @@ -582,14 +612,14 @@ * @param currMapJoinOp * the currMapJoinOp to set */ - public void setCurrMapJoinOp(MapJoinOperator currMapJoinOp) { + public void setCurrMapJoinOp(AbstractMapJoinOperator currMapJoinOp) { this.currMapJoinOp = currMapJoinOp; } /** * @return the listRejectedMapJoins */ - public List getListRejectedMapJoins() { + public List> getListRejectedMapJoins() { return listRejectedMapJoins; } @@ -598,7 +628,7 @@ * the listRejectedMapJoins to set */ public void setListRejectedMapJoins( - List listRejectedMapJoins) { + List> listRejectedMapJoins) { this.listRejectedMapJoins = listRejectedMapJoins; } } Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java (working copy) @@ -58,6 +58,9 @@ transformations.add(new MapJoinProcessor()); if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN)) { transformations.add(new BucketMapJoinOptimizer()); + if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)) { + transformations.add(new SortedMergeBucketMapJoinOptimizer()); + } } transformations.add(new UnionProcessor()); transformations.add(new JoinReorder()); Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeBucketMapJoinOptimizer.java (revision 0) @@ -0,0 +1,262 @@ +/** + * 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; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.exec.MapJoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; +import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; +import org.apache.hadoop.hive.ql.lib.Dispatcher; +import org.apache.hadoop.hive.ql.lib.GraphWalker; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.lib.Rule; +import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner; +import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; +import org.apache.hadoop.hive.ql.parse.QBJoinTree; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; +import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; + +//try to replace a bucket map join with a sorted merge map join +public class SortedMergeBucketMapJoinOptimizer implements Transform { + + private static final Log LOG = LogFactory + .getLog(SortedMergeBucketMapJoinOptimizer.class.getName()); + + public SortedMergeBucketMapJoinOptimizer() { + } + + @Override + public ParseContext transform(ParseContext pctx) throws SemanticException { + + Map opRules = new LinkedHashMap(); + // go through all map joins and find out all which have enabled bucket map + // join. + opRules.put(new RuleRegExp("R1", "MAPJOIN%"), + getSortedMergeBucketMapjoinProc(pctx)); + // The dispatcher fires the processor corresponding to the closest matching + // rule and passes the context along + Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, null); + GraphWalker ogw = new DefaultGraphWalker(disp); + + // Create a list of topop nodes + ArrayList topNodes = new ArrayList(); + topNodes.addAll(pctx.getTopOps().values()); + ogw.startWalking(topNodes, null); + + return pctx; + } + + private NodeProcessor getSortedMergeBucketMapjoinProc(ParseContext pctx) { + return new SortedMergeBucketMapjoinProc(pctx); + } + + private NodeProcessor getDefaultProc() { + return new NodeProcessor() { + @Override + public Object process(Node nd, Stack stack, + NodeProcessorCtx procCtx, Object... nodeOutputs) + throws SemanticException { + return null; + } + }; + } + + class SortedMergeBucketMapjoinProc implements NodeProcessor { + ParseContext pGraphContext; + + public SortedMergeBucketMapjoinProc(ParseContext pctx) { + this.pGraphContext = pctx; + } + + public SortedMergeBucketMapjoinProc() { + } + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + if (nd instanceof SMBMapJoinOperator) { + return null; + } + MapJoinOperator mapJoinOp = (MapJoinOperator) nd; + if (mapJoinOp.getConf().getAliasBucketFileNameMapping() == null + || mapJoinOp.getConf().getAliasBucketFileNameMapping().size() == 0) { + return null; + } + + boolean tableSorted = true; + QBJoinTree joinCxt = this.pGraphContext.getMapJoinContext() + .get(mapJoinOp); + if (joinCxt == null) + return null; + String[] srcs = joinCxt.getBaseSrc(); + int pos = 0; + for (String src : srcs) { + tableSorted = tableSorted + && isTableSorted(this.pGraphContext, mapJoinOp, joinCxt, src, pos); + pos++; + } + if (!tableSorted) { + //this is a mapjoin but not suit for a sort merge bucket map join. check outer joins + MapJoinProcessor.checkMapJoin(((MapJoinOperator) nd).getConf().getPosBigTable(), + ((MapJoinOperator) nd).getConf().getConds()); + return null; + } + // convert a bucket map join operator to a sorted merge bucket map join + // operator + convertToSMBJoin(mapJoinOp, srcs); + return null; + } + + private SMBMapJoinOperator convertToSMBJoin(MapJoinOperator mapJoinOp, + String[] srcs) { + SMBMapJoinOperator smbJop = new SMBMapJoinOperator(mapJoinOp); + SMBJoinDesc smbJoinDesc = new SMBJoinDesc(mapJoinOp.getConf()); + smbJop.setConf(smbJoinDesc); + HashMap tagToAlias = new HashMap(); + for (int i = 0; i < srcs.length; i++) { + tagToAlias.put((byte) i, srcs[i]); + } + smbJoinDesc.setTagToAlias(tagToAlias); + + int indexInListMapJoinNoReducer = this.pGraphContext.getListMapJoinOpsNoReducer().indexOf(mapJoinOp); + if(indexInListMapJoinNoReducer >= 0 ) { + this.pGraphContext.getListMapJoinOpsNoReducer().remove(indexInListMapJoinNoReducer); + this.pGraphContext.getListMapJoinOpsNoReducer().add(indexInListMapJoinNoReducer, smbJop); + } + + List parentOperators = mapJoinOp.getParentOperators(); + for (int i = 0; i < parentOperators.size(); i++) { + Operator par = parentOperators.get(i); + int index = par.getChildOperators().indexOf(mapJoinOp); + par.getChildOperators().remove(index); + par.getChildOperators().add(index, smbJop); + } + List childOps = mapJoinOp.getChildOperators(); + for (int i = 0; i < childOps.size(); i++) { + Operator child = childOps.get(i); + int index = child.getParentOperators().indexOf(mapJoinOp); + child.getParentOperators().remove(index); + child.getParentOperators().add(index, smbJop); + } + return smbJop; + } + + private boolean isTableSorted(ParseContext pctx, MapJoinOperator op, + QBJoinTree joinTree, String alias, int pos) throws SemanticException { + Map> topOps = this.pGraphContext + .getTopOps(); + Map topToTable = this.pGraphContext + .getTopToTable(); + TableScanOperator tso = (TableScanOperator) topOps.get(alias); + if (tso == null) + return false; + + List keys = op.getConf().getKeys().get((byte) pos); + // get all join columns from join keys stored in MapJoinDesc + List joinCols = new ArrayList(); + List joinKeys = new ArrayList(); + joinKeys.addAll(keys); + while (joinKeys.size() > 0) { + ExprNodeDesc node = joinKeys.remove(0); + if (node instanceof ExprNodeColumnDesc) { + joinCols.addAll(node.getCols()); + } else if (node instanceof ExprNodeGenericFuncDesc) { + ExprNodeGenericFuncDesc udfNode = ((ExprNodeGenericFuncDesc) node); + GenericUDF udf = udfNode.getGenericUDF(); + if (!FunctionRegistry.isDeterministic(udf)) { + return false; + } + joinKeys.addAll(0, udfNode.getChildExprs()); + } + } + + Table tbl = topToTable.get(tso); + if (tbl.isPartitioned()) { + PrunedPartitionList prunedParts = null; + try { + prunedParts = PartitionPruner.prune(tbl, pGraphContext + .getOpToPartPruner().get(tso), pGraphContext.getConf(), alias, + pGraphContext.getPrunedPartitions()); + } catch (HiveException e) { + LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e)); + throw new SemanticException(e.getMessage(), e); + } + boolean ret = true; + for (Partition p : prunedParts.getConfirmedPartns()) { + ret = ret && checkSortColsAndJoinCols(p.getSortCols(), joinCols); + if (!ret) { + return false; + } + } + for (Partition p : prunedParts.getUnknownPartns()) { + ret = ret && checkSortColsAndJoinCols(p.getSortCols(), joinCols); + if (!ret) { + return false; + } + } + } else { + return checkSortColsAndJoinCols(tbl.getSortCols(), joinCols); + } + return true; + } + + private boolean checkSortColsAndJoinCols(List sortCols, + List joinCols) { + // require all sort columns are asc, right now only support asc + List sortColNames = new ArrayList(); + for (Order o : sortCols) { + if (o.getOrder() != BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC) { + return false; + } + sortColNames.add(o.getCol()); + } + + return sortColNames.containsAll(joinCols) + && sortColNames.size() == joinCols.size(); + } + } + +} Index: ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java (working copy) @@ -75,7 +75,10 @@ protected Context ctx; protected HashMap idToTableNameMap; - + + public static int HIVE_COLUMN_ORDER_ASC = 1; + public static int HIVE_COLUMN_ORDER_DESC = 0; + /** * ReadEntitites that are passed to the hooks. */ @@ -355,10 +358,10 @@ ASTNode child = (ASTNode) ast.getChild(i); if (child.getToken().getType() == HiveParser.TOK_TABSORTCOLNAMEASC) { colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()), - 1)); + HIVE_COLUMN_ORDER_ASC)); } else { colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()), - 0)); + HIVE_COLUMN_ORDER_DESC)); } } return colList; Index: ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java (working copy) @@ -27,6 +27,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.GroupByOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; @@ -37,6 +38,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.LoadFileDesc; import org.apache.hadoop.hive.ql.plan.LoadTableDesc; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.FilterDesc.sampleDesc; /** @@ -67,7 +69,7 @@ private HashMap idToTableNameMap; private int destTableId; private UnionProcContext uCtx; - private List listMapJoinOpsNoReducer; // list of map join + private List> listMapJoinOpsNoReducer; // list of map join // operators with no // reducer private Map> groupOpToInputTables; @@ -129,7 +131,7 @@ HashMap topToTable, List loadTableWork, List loadFileWork, Context ctx, HashMap idToTableNameMap, int destTableId, - UnionProcContext uCtx, List listMapJoinOpsNoReducer, + UnionProcContext uCtx, List> listMapJoinOpsNoReducer, Map> groupOpToInputTables, Map prunedPartitions, HashMap opToSamplePruner) { @@ -366,7 +368,7 @@ /** * @return the listMapJoinOpsNoReducer */ - public List getListMapJoinOpsNoReducer() { + public List> getListMapJoinOpsNoReducer() { return listMapJoinOpsNoReducer; } @@ -375,7 +377,7 @@ * the listMapJoinOpsNoReducer to set */ public void setListMapJoinOpsNoReducer( - List listMapJoinOpsNoReducer) { + List> listMapJoinOpsNoReducer) { this.listMapJoinOpsNoReducer = listMapJoinOpsNoReducer; } Index: ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (revision 917282) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (working copy) @@ -45,6 +45,7 @@ import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.exec.ConditionalTask; import org.apache.hadoop.hive.ql.exec.ExecDriver; @@ -123,6 +124,7 @@ import org.apache.hadoop.hive.ql.plan.LimitDesc; import org.apache.hadoop.hive.ql.plan.LoadFileDesc; import org.apache.hadoop.hive.ql.plan.LoadTableDesc; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; @@ -173,7 +175,7 @@ private ASTNode ast; private int destTableId; private UnionProcContext uCtx; - List listMapJoinOpsNoReducer; + List> listMapJoinOpsNoReducer; private HashMap opToSamplePruner; Map> groupOpToInputTables; Map prunedPartitions; @@ -202,7 +204,7 @@ topToTable = new HashMap(); destTableId = 1; uCtx = null; - listMapJoinOpsNoReducer = new ArrayList(); + listMapJoinOpsNoReducer = new ArrayList>(); groupOpToInputTables = new HashMap>(); prunedPartitions = new HashMap(); unparseTranslator = new UnparseTranslator(); Index: ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java (working copy) @@ -89,6 +89,21 @@ this(exprs, outputColumnNames, false, conds); } + public JoinDesc(JoinDesc clone) { + this.bigKeysDirMap = clone.bigKeysDirMap; + this.conds = clone.conds; + this.exprs = clone.exprs; + this.handleSkewJoin = clone.handleSkewJoin; + this.keyTableDesc = clone.keyTableDesc; + this.noOuterJoin = clone.noOuterJoin; + this.outputColumnNames = clone.outputColumnNames; + this.reversedExprs = clone.reversedExprs; + this.skewKeyDefinition = clone.skewKeyDefinition; + this.skewKeysValuesTables = clone.skewKeysValuesTables; + this.smallKeysDirMap = clone.smallKeysDirMap; + this.tagOrder = clone.tagOrder; + } + public Map> getExprs() { return exprs; } Index: ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java (revision 916496) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java (working copy) @@ -50,6 +50,17 @@ public MapJoinDesc() { } + + public MapJoinDesc(MapJoinDesc clone) { + super(clone); + this.keys = clone.keys; + this.keyTblDesc = clone.keyTblDesc; + this.valueTblDescs = clone.valueTblDescs; + this.posBigTable = clone.posBigTable; + this.retainList = clone.retainList; + this.bigTableAlias = clone.bigTableAlias; + this.aliasBucketFileNameMapping = clone.aliasBucketFileNameMapping; + } public MapJoinDesc(final Map> keys, final TableDesc keyTblDesc, final Map> values, Index: ql/src/java/org/apache/hadoop/hive/ql/plan/SMBJoinDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/SMBJoinDesc.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/SMBJoinDesc.java (revision 0) @@ -0,0 +1,57 @@ +/** + * 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.plan; + +import java.io.Serializable; +import java.util.HashMap; + +@Explain(displayName = "Sorted Merge Bucket Map Join Operator") +public class SMBJoinDesc extends MapJoinDesc implements Serializable { + + private static final long serialVersionUID = 1L; + + private MapredLocalWork localWork; + + //keep a mapping from tag to the fetch operator alias + private HashMap tagToAlias; + + public SMBJoinDesc(MapJoinDesc conf) { + super(conf); + } + + public SMBJoinDesc() { + } + + public MapredLocalWork getLocalWork() { + return localWork; + } + + public void setLocalWork(MapredLocalWork localWork) { + this.localWork = localWork; + } + + public HashMap getTagToAlias() { + return tagToAlias; + } + + public void setTagToAlias(HashMap tagToAlias) { + this.tagToAlias = tagToAlias; + } + +} Index: ql/src/test/queries/clientnegative/smb_bucketmapjoin.q =================================================================== --- ql/src/test/queries/clientnegative/smb_bucketmapjoin.q (revision 0) +++ ql/src/test/queries/clientnegative/smb_bucketmapjoin.q (revision 0) @@ -0,0 +1,23 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +drop table smb_bucket4_1; +CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; + +drop table smb_bucket4_2; +CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; + +insert overwrite table smb_bucket4_1 +select * from src; + +insert overwrite table smb_bucket4_2 +select * from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +select /*+mapjoin(a)*/ * from smb_bucket4_1 a left outer join smb_bucket4_2 b on a.key = b.key; + +drop table smb_bucket4_1; +drop table smb_bucket4_2; Index: ql/src/test/queries/clientpositive/smb_mapjoin_1.q =================================================================== --- ql/src/test/queries/clientpositive/smb_mapjoin_1.q (revision 0) +++ ql/src/test/queries/clientpositive/smb_mapjoin_1.q (revision 0) @@ -0,0 +1,52 @@ +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1; +load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2; +load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key; + + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key; + + +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; Index: ql/src/test/queries/clientpositive/smb_mapjoin_2.q =================================================================== --- ql/src/test/queries/clientpositive/smb_mapjoin_2.q (revision 0) +++ ql/src/test/queries/clientpositive/smb_mapjoin_2.q (revision 0) @@ -0,0 +1,52 @@ +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1; +load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2; +load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key; + + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key; + + +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; Index: ql/src/test/queries/clientpositive/smb_mapjoin_3.q =================================================================== --- ql/src/test/queries/clientpositive/smb_mapjoin_3.q (revision 0) +++ ql/src/test/queries/clientpositive/smb_mapjoin_3.q (revision 0) @@ -0,0 +1,52 @@ +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1; +load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2; +load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key; + + +explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key; + + +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; Index: ql/src/test/queries/clientpositive/smb_mapjoin_4.q =================================================================== --- ql/src/test/queries/clientpositive/smb_mapjoin_4.q (revision 0) +++ ql/src/test/queries/clientpositive/smb_mapjoin_4.q (revision 0) @@ -0,0 +1,71 @@ +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1; +load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2; +load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + + +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; Index: ql/src/test/queries/clientpositive/smb_mapjoin_5.q =================================================================== --- ql/src/test/queries/clientpositive/smb_mapjoin_5.q (revision 0) +++ ql/src/test/queries/clientpositive/smb_mapjoin_5.q (revision 0) @@ -0,0 +1,71 @@ +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; + +create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; +create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; + +load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1; +load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2; +load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key; + +explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key; + + +drop table smb_bucket_3; +drop table smb_bucket_2; +drop table smb_bucket_1; Index: ql/src/test/queries/clientpositive/smb_mapjoin_6.q =================================================================== --- ql/src/test/queries/clientpositive/smb_mapjoin_6.q (revision 0) +++ ql/src/test/queries/clientpositive/smb_mapjoin_6.q (revision 0) @@ -0,0 +1,43 @@ +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +set hive.exec.reducers.max = 1; + +drop table smb_bucket4_1; +CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +drop table smb_bucket4_2; +CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +insert overwrite table smb_bucket4_1 +select * from src; + +insert overwrite table smb_bucket4_2 +select * from src; + +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +explain +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; + +explain +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key; + + +explain +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000; +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000; + + +explain +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000; +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000; + +explain +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key join smb_bucket4_2 c on b.key = c.key where a.key>1000; +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key join smb_bucket4_2 c on b.key = c.key where a.key>1000; + +drop table smb_bucket4_1; +drop table smb_bucket4_2; Index: ql/src/test/results/clientnegative/smb_bucketmapjoin.q.out =================================================================== --- ql/src/test/results/clientnegative/smb_bucketmapjoin.q.out (revision 0) +++ ql/src/test/results/clientnegative/smb_bucketmapjoin.q.out (revision 0) @@ -0,0 +1,39 @@ +PREHOOK: query: drop table smb_bucket4_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket4_1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket4_1 +PREHOOK: query: drop table smb_bucket4_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket4_2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket4_2 +PREHOOK: query: insert overwrite table smb_bucket4_1 +select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 +select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@smb_bucket4_1 +PREHOOK: query: insert overwrite table smb_bucket4_2 +select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 +select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@smb_bucket4_2 +FAILED: Error in semantic analysis: Map Join cannot be performed with Outer join Index: ql/src/test/results/clientpositive/smb_mapjoin_1.q.out =================================================================== --- ql/src/test/results/clientpositive/smb_mapjoin_1.q.out (revision 0) +++ ql/src/test/results/clientpositive/smb_mapjoin_1.q.out (revision 0) @@ -0,0 +1,701 @@ +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-19_882_7459646971783867464/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-19_882_7459646971783867464/10000 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-29_792_3264771212721856103/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-29_792_3264771212721856103/10000 +1 val_1 NULL NULL +3 val_3 NULL NULL +4 val_4 NULL NULL +5 val_5 NULL NULL +10 val_10 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-38_587_6314921126722110687/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-38_587_6314921126722110687/10000 +NULL NULL 20 val_20 +NULL NULL 23 val_23 +NULL NULL 25 val_25 +NULL NULL 30 val_30 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-45_495_1242835629584726058/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-45_495_1242835629584726058/10000 +1 val_1 NULL NULL +3 val_3 NULL NULL +4 val_4 NULL NULL +5 val_5 NULL NULL +10 val_10 NULL NULL +NULL NULL 20 val_20 +NULL NULL 23 val_23 +NULL NULL 25 val_25 +NULL NULL 30 val_30 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-50_444_4171021467210511233/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-50_444_4171021467210511233/10000 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-56_335_4306785338247643641/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-09-56_335_4306785338247643641/10000 +1 val_1 NULL NULL +3 val_3 NULL NULL +4 val_4 NULL NULL +5 val_5 NULL NULL +10 val_10 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-01_016_3515998402025465674/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-01_016_3515998402025465674/10000 +NULL NULL 20 val_20 +NULL NULL 23 val_23 +NULL NULL 25 val_25 +NULL NULL 30 val_30 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-06_086_690383684056454245/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-06_086_690383684056454245/10000 +1 val_1 NULL NULL +3 val_3 NULL NULL +4 val_4 NULL NULL +5 val_5 NULL NULL +10 val_10 NULL NULL +NULL NULL 20 val_20 +NULL NULL 23 val_23 +NULL NULL 25 val_25 +NULL NULL 30 val_30 +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_1 Index: ql/src/test/results/clientpositive/smb_mapjoin_2.q.out =================================================================== --- ql/src/test/results/clientpositive/smb_mapjoin_2.q.out (revision 0) +++ ql/src/test/results/clientpositive/smb_mapjoin_2.q.out (revision 0) @@ -0,0 +1,709 @@ +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_3 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-14_473_8620838922433877276/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-14_473_8620838922433877276/10000 +4 val_4 4 val_4 +10 val_10 10 val_10 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_3 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-21_032_892659655699611140/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-21_032_892659655699611140/10000 +1 val_1 NULL NULL +3 val_3 NULL NULL +4 val_4 4 val_4 +5 val_5 NULL NULL +10 val_10 10 val_10 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_3 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-26_629_4547984603395735026/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-26_629_4547984603395735026/10000 +4 val_4 4 val_4 +10 val_10 10 val_10 +NULL NULL 17 val_17 +NULL NULL 19 val_19 +NULL NULL 20 val_20 +NULL NULL 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_3 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-31_646_592121791265422282/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-31_646_592121791265422282/10000 +1 val_1 NULL NULL +3 val_3 NULL NULL +4 val_4 4 val_4 +5 val_5 NULL NULL +10 val_10 10 val_10 +NULL NULL 17 val_17 +NULL NULL 19 val_19 +NULL NULL 20 val_20 +NULL NULL 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_3 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-36_604_8352065342128674850/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-36_604_8352065342128674850/10000 +4 val_4 4 val_4 +10 val_10 10 val_10 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_3 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-42_035_301552626761233730/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a left outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-42_035_301552626761233730/10000 +1 val_1 NULL NULL +3 val_3 NULL NULL +4 val_4 4 val_4 +5 val_5 NULL NULL +10 val_10 10 val_10 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_3 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-49_566_5694296943126276110/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a right outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-49_566_5694296943126276110/10000 +4 val_4 4 val_4 +10 val_10 10 val_10 +NULL NULL 17 val_17 +NULL NULL 19 val_19 +NULL NULL 20 val_20 +NULL NULL 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_3 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-54_036_179984459526613835/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_1 a full outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-10-54_036_179984459526613835/10000 +1 val_1 NULL NULL +3 val_3 NULL NULL +4 val_4 4 val_4 +5 val_5 NULL NULL +10 val_10 10 val_10 +NULL NULL 17 val_17 +NULL NULL 19 val_19 +NULL NULL 20 val_20 +NULL NULL 23 val_23 +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_1 Index: ql/src/test/results/clientpositive/smb_mapjoin_3.q.out =================================================================== --- ql/src/test/results/clientpositive/smb_mapjoin_3.q.out (revision 0) +++ ql/src/test/results/clientpositive/smb_mapjoin_3.q.out (revision 0) @@ -0,0 +1,705 @@ +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket_2 a) (TOK_TABREF smb_bucket_3 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-02_158_5767278301144578948/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-02_158_5767278301144578948/10000 +20 val_20 20 val_20 +23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_2 a) (TOK_TABREF smb_bucket_3 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-07_089_1881480579157421362/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-07_089_1881480579157421362/10000 +20 val_20 20 val_20 +23 val_23 23 val_23 +25 val_25 NULL NULL +30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_2 a) (TOK_TABREF smb_bucket_3 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-14_232_3797331809795003304/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-14_232_3797331809795003304/10000 +NULL NULL 4 val_4 +NULL NULL 10 val_10 +NULL NULL 17 val_17 +NULL NULL 19 val_19 +20 val_20 20 val_20 +23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_2 a) (TOK_TABREF smb_bucket_3 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-20_421_1118024942572143359/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-20_421_1118024942572143359/10000 +NULL NULL 4 val_4 +NULL NULL 10 val_10 +NULL NULL 17 val_17 +NULL NULL 19 val_19 +20 val_20 20 val_20 +23 val_23 23 val_23 +25 val_25 NULL NULL +30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket_2 a) (TOK_TABREF smb_bucket_3 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-25_217_3201115310663607412/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_2 a join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-25_217_3201115310663607412/10000 +20 val_20 20 val_20 +23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_2 a) (TOK_TABREF smb_bucket_3 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-29_696_2352663068233909459/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_2 a left outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-29_696_2352663068233909459/10000 +20 val_20 20 val_20 +23 val_23 23 val_23 +25 val_25 NULL NULL +30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_2 a) (TOK_TABREF smb_bucket_3 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-35_258_2220511898157344609/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_2 a right outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-35_258_2220511898157344609/10000 +NULL NULL 4 val_4 +NULL NULL 10 val_10 +NULL NULL 17 val_17 +NULL NULL 19 val_19 +20 val_20 20 val_20 +23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_2 a) (TOK_TABREF smb_bucket_3 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-41_120_7615655449182462424/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket_2 a full outer join smb_bucket_3 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_15-11-41_120_7615655449182462424/10000 +NULL NULL 4 val_4 +NULL NULL 10 val_10 +NULL NULL 17 val_17 +NULL NULL 19 val_19 +20 val_20 20 val_20 +23 val_23 23 val_23 +25 val_25 NULL NULL +30 val_30 NULL NULL +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_1 Index: ql/src/test/results/clientpositive/smb_mapjoin_4.q.out =================================================================== --- ql/src/test/results/clientpositive/smb_mapjoin_4.q.out (revision 0) +++ ql/src/test/results/clientpositive/smb_mapjoin_4.q.out (revision 0) @@ -0,0 +1,1286 @@ +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-00_210_1829923077226292707/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-00_210_1829923077226292707/10000 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-06_152_5173214839622756029/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-06_152_5173214839622756029/10000 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-11_033_8583768958398984844/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-11_033_8583768958398984844/10000 +1 val_1 NULL NULL NULL NULL +3 val_3 NULL NULL NULL NULL +4 val_4 NULL NULL NULL NULL +5 val_5 NULL NULL NULL NULL +10 val_10 NULL NULL NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-16_573_7787391547742353628/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-16_573_7787391547742353628/10000 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL NULL NULL 20 val_20 +NULL NULL NULL NULL 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Outer Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-21_601_751472388328632390/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-21_601_751472388328632390/10000 +1 val_1 NULL NULL NULL NULL +3 val_3 NULL NULL NULL NULL +4 val_4 NULL NULL NULL NULL +NULL NULL NULL NULL 4 val_4 +5 val_5 NULL NULL NULL NULL +10 val_10 NULL NULL NULL NULL +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL NULL NULL 20 val_20 +NULL NULL NULL NULL 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-27_807_5895128564531534688/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-27_807_5895128564531534688/10000 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-34_621_392380752222393807/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-34_621_392380752222393807/10000 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +NULL NULL 25 val_25 NULL NULL +NULL NULL 30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-39_498_7247094736430991660/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-39_498_7247094736430991660/10000 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + Outer Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-44_431_7704143536154480996/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-44_431_7704143536154480996/10000 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +NULL NULL 25 val_25 NULL NULL +NULL NULL 30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-48_951_5922101659780969180/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-48_951_5922101659780969180/10000 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-53_486_5129994662382156685/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-53_486_5129994662382156685/10000 +1 val_1 NULL NULL NULL NULL +3 val_3 NULL NULL NULL NULL +4 val_4 NULL NULL NULL NULL +5 val_5 NULL NULL NULL NULL +10 val_10 NULL NULL NULL NULL +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +NULL NULL 25 val_25 NULL NULL +NULL NULL 30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-58_116_6789446490168521813/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-49-58_116_6789446490168521813/10000 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + c + TableScan + alias: c + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + Outer Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-50-02_988_3713838872570451484/10000 +POSTHOOK: query: select /*+mapjoin(a,b)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_20-50-02_988_3713838872570451484/10000 +1 val_1 NULL NULL NULL NULL +3 val_3 NULL NULL NULL NULL +4 val_4 NULL NULL NULL NULL +NULL NULL NULL NULL 4 val_4 +5 val_5 NULL NULL NULL NULL +10 val_10 NULL NULL NULL NULL +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +NULL NULL 25 val_25 NULL NULL +NULL NULL 30 val_30 NULL NULL +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_1 Index: ql/src/test/results/clientpositive/smb_mapjoin_5.q.out =================================================================== --- ql/src/test/results/clientpositive/smb_mapjoin_5.q.out (revision 0) +++ ql/src/test/results/clientpositive/smb_mapjoin_5.q.out (revision 0) @@ -0,0 +1,1286 @@ +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_1.txt' overwrite into table smb_bucket_1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_1 +PREHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_2.txt' overwrite into table smb_bucket_2 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +PREHOOK: type: LOAD +POSTHOOK: query: load data local inpath '../data/files/smbbucket_3.txt' overwrite into table smb_bucket_3 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-01_611_1843701384382069624/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-01_611_1843701384382069624/10000 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-08_054_3080420924189631670/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-08_054_3080420924189631670/10000 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-12_898_7324368041397074660/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-12_898_7324368041397074660/10000 +1 val_1 NULL NULL NULL NULL +3 val_3 NULL NULL NULL NULL +4 val_4 NULL NULL NULL NULL +5 val_5 NULL NULL NULL NULL +10 val_10 NULL NULL NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-18_928_9207467659300900566/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-18_928_9207467659300900566/10000 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL NULL NULL 20 val_20 +NULL NULL NULL NULL 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Left Outer Join0 to 1 + Outer Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-24_485_7059595195278880284/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-24_485_7059595195278880284/10000 +1 val_1 NULL NULL NULL NULL +3 val_3 NULL NULL NULL NULL +4 val_4 NULL NULL NULL NULL +NULL NULL NULL NULL 4 val_4 +5 val_5 NULL NULL NULL NULL +10 val_10 NULL NULL NULL NULL +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL NULL NULL 20 val_20 +NULL NULL NULL NULL 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-30_727_981768489075827678/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-30_727_981768489075827678/10000 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-36_646_2352379748748388100/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-36_646_2352379748748388100/10000 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +NULL NULL 25 val_25 NULL NULL +NULL NULL 30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-42_218_6449196226642922552/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-42_218_6449196226642922552/10000 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Right Outer Join0 to 1 + Outer Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-48_023_4076028328539635057/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a right outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-48_023_4076028328539635057/10000 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +NULL NULL 25 val_25 NULL NULL +NULL NULL 30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-53_120_6546943353558446747/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-53_120_6546943353558446747/10000 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-57_815_4348689831088701309/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key left outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-32-57_815_4348689831088701309/10000 +1 val_1 NULL NULL NULL NULL +3 val_3 NULL NULL NULL NULL +4 val_4 NULL NULL NULL NULL +5 val_5 NULL NULL NULL NULL +10 val_10 NULL NULL NULL NULL +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +NULL NULL 25 val_25 NULL NULL +NULL NULL 30 val_30 NULL NULL +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-33-03_374_1599987657628909567/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key right outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-33-03_374_1599987657628909567/10000 +NULL NULL NULL NULL 4 val_4 +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +PREHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_FULLOUTERJOIN (TOK_TABREF smb_bucket_1 a) (TOK_TABREF smb_bucket_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket_3 c) (= (. (TOK_TABLE_OR_COL b) 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 c))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Outer Join 0 to 1 + Outer Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket_2 +PREHOOK: Input: default@smb_bucket_3 +PREHOOK: Input: default@smb_bucket_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-33-09_592_668458787004135220/10000 +POSTHOOK: query: select /*+mapjoin(a,c)*/ * from smb_bucket_1 a full outer join smb_bucket_2 b on a.key = b.key full outer join smb_bucket_3 c on b.key=c.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket_2 +POSTHOOK: Input: default@smb_bucket_3 +POSTHOOK: Input: default@smb_bucket_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-02_16-33-09_592_668458787004135220/10000 +1 val_1 NULL NULL NULL NULL +3 val_3 NULL NULL NULL NULL +4 val_4 NULL NULL NULL NULL +NULL NULL NULL NULL 4 val_4 +5 val_5 NULL NULL NULL NULL +10 val_10 NULL NULL NULL NULL +NULL NULL NULL NULL 10 val_10 +NULL NULL NULL NULL 17 val_17 +NULL NULL NULL NULL 19 val_19 +NULL NULL 20 val_20 20 val_20 +NULL NULL 23 val_23 23 val_23 +NULL NULL 25 val_25 NULL NULL +NULL NULL 30 val_30 NULL NULL +PREHOOK: query: drop table smb_bucket_3 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_3 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_3 +PREHOOK: query: drop table smb_bucket_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_2 +PREHOOK: query: drop table smb_bucket_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket_1 Index: ql/src/test/results/clientpositive/smb_mapjoin_6.q.out =================================================================== --- ql/src/test/results/clientpositive/smb_mapjoin_6.q.out (revision 0) +++ ql/src/test/results/clientpositive/smb_mapjoin_6.q.out (revision 0) @@ -0,0 +1,2513 @@ +PREHOOK: query: drop table smb_bucket4_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket4_1 +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket4_1 +PREHOOK: query: drop table smb_bucket4_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket4_2 +POSTHOOK: type: DROPTABLE +PREHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@smb_bucket4_2 +PREHOOK: query: insert overwrite table smb_bucket4_1 +select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@smb_bucket4_1 +POSTHOOK: query: insert overwrite table smb_bucket4_1 +select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@smb_bucket4_1 +PREHOOK: query: insert overwrite table smb_bucket4_2 +select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@smb_bucket4_2 +POSTHOOK: query: insert overwrite table smb_bucket4_2 +select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@smb_bucket4_2 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket4_1 a) (TOK_TABREF smb_bucket4_2 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_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-25_886_3394497494124396512/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-25_886_3394497494124396512/10000 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +2 val_2 2 val_2 +4 val_4 4 val_4 +8 val_8 8 val_8 +10 val_10 10 val_10 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +20 val_20 20 val_20 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +28 val_28 28 val_28 +30 val_30 30 val_30 +34 val_34 34 val_34 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +44 val_44 44 val_44 +54 val_54 54 val_54 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +64 val_64 64 val_64 +66 val_66 66 val_66 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +74 val_74 74 val_74 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +78 val_78 78 val_78 +80 val_80 80 val_80 +82 val_82 82 val_82 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +86 val_86 86 val_86 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +92 val_92 92 val_92 +96 val_96 96 val_96 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +114 val_114 114 val_114 +116 val_116 116 val_116 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +126 val_126 126 val_126 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +136 val_136 136 val_136 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +156 val_156 156 val_156 +158 val_158 158 val_158 +160 val_160 160 val_160 +162 val_162 162 val_162 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +166 val_166 166 val_166 +168 val_168 168 val_168 +170 val_170 170 val_170 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +178 val_178 178 val_178 +180 val_180 180 val_180 +186 val_186 186 val_186 +190 val_190 190 val_190 +192 val_192 192 val_192 +194 val_194 194 val_194 +196 val_196 196 val_196 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +202 val_202 202 val_202 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +214 val_214 214 val_214 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +218 val_218 218 val_218 +222 val_222 222 val_222 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +226 val_226 226 val_226 +228 val_228 228 val_228 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +244 val_244 244 val_244 +248 val_248 248 val_248 +252 val_252 252 val_252 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +258 val_258 258 val_258 +260 val_260 260 val_260 +262 val_262 262 val_262 +266 val_266 266 val_266 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +274 val_274 274 val_274 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +284 val_284 284 val_284 +286 val_286 286 val_286 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +292 val_292 292 val_292 +296 val_296 296 val_296 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +302 val_302 302 val_302 +306 val_306 306 val_306 +308 val_308 308 val_308 +310 val_310 310 val_310 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +332 val_332 332 val_332 +336 val_336 336 val_336 +338 val_338 338 val_338 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +356 val_356 356 val_356 +360 val_360 360 val_360 +362 val_362 362 val_362 +364 val_364 364 val_364 +366 val_366 366 val_366 +368 val_368 368 val_368 +374 val_374 374 val_374 +378 val_378 378 val_378 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +386 val_386 386 val_386 +392 val_392 392 val_392 +394 val_394 394 val_394 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +400 val_400 400 val_400 +402 val_402 402 val_402 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +418 val_418 418 val_418 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +432 val_432 432 val_432 +436 val_436 436 val_436 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +444 val_444 444 val_444 +446 val_446 446 val_446 +448 val_448 448 val_448 +452 val_452 452 val_452 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +460 val_460 460 val_460 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +470 val_470 470 val_470 +472 val_472 472 val_472 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +482 val_482 482 val_482 +484 val_484 484 val_484 +490 val_490 490 val_490 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +494 val_494 494 val_494 +496 val_496 496 val_496 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +9 val_9 9 val_9 +11 val_11 11 val_11 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +17 val_17 17 val_17 +19 val_19 19 val_19 +27 val_27 27 val_27 +33 val_33 33 val_33 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +41 val_41 41 val_41 +43 val_43 43 val_43 +47 val_47 47 val_47 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +53 val_53 53 val_53 +57 val_57 57 val_57 +65 val_65 65 val_65 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +69 val_69 69 val_69 +77 val_77 77 val_77 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +85 val_85 85 val_85 +87 val_87 87 val_87 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +105 val_105 105 val_105 +111 val_111 111 val_111 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +131 val_131 131 val_131 +133 val_133 133 val_133 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +143 val_143 143 val_143 +145 val_145 145 val_145 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +153 val_153 153 val_153 +155 val_155 155 val_155 +157 val_157 157 val_157 +163 val_163 163 val_163 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +177 val_177 177 val_177 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +181 val_181 181 val_181 +183 val_183 183 val_183 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +189 val_189 189 val_189 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +201 val_201 201 val_201 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +235 val_235 235 val_235 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +241 val_241 241 val_241 +247 val_247 247 val_247 +249 val_249 249 val_249 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +257 val_257 257 val_257 +263 val_263 263 val_263 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +275 val_275 275 val_275 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +283 val_283 283 val_283 +285 val_285 285 val_285 +287 val_287 287 val_287 +289 val_289 289 val_289 +291 val_291 291 val_291 +305 val_305 305 val_305 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +315 val_315 315 val_315 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +323 val_323 323 val_323 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +335 val_335 335 val_335 +339 val_339 339 val_339 +341 val_341 341 val_341 +345 val_345 345 val_345 +351 val_351 351 val_351 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +365 val_365 365 val_365 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +373 val_373 373 val_373 +375 val_375 375 val_375 +377 val_377 377 val_377 +379 val_379 379 val_379 +389 val_389 389 val_389 +393 val_393 393 val_393 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +407 val_407 407 val_407 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +411 val_411 411 val_411 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +419 val_419 419 val_419 +421 val_421 421 val_421 +427 val_427 427 val_427 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +435 val_435 435 val_435 +437 val_437 437 val_437 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +443 val_443 443 val_443 +449 val_449 449 val_449 +453 val_453 453 val_453 +455 val_455 455 val_455 +457 val_457 457 val_457 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +467 val_467 467 val_467 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +475 val_475 475 val_475 +477 val_477 477 val_477 +479 val_479 479 val_479 +481 val_481 481 val_481 +483 val_483 483 val_483 +485 val_485 485 val_485 +487 val_487 487 val_487 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +491 val_491 491 val_491 +493 val_493 493 val_493 +495 val_495 495 val_495 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket4_1 a) (TOK_TABREF smb_bucket4_2 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-30_611_4743434373328164978/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-30_611_4743434373328164978/10000 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +0 val_0 0 val_0 +2 val_2 2 val_2 +4 val_4 4 val_4 +8 val_8 8 val_8 +10 val_10 10 val_10 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +12 val_12 12 val_12 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +18 val_18 18 val_18 +20 val_20 20 val_20 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +24 val_24 24 val_24 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +26 val_26 26 val_26 +28 val_28 28 val_28 +30 val_30 30 val_30 +34 val_34 34 val_34 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +42 val_42 42 val_42 +44 val_44 44 val_44 +54 val_54 54 val_54 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +58 val_58 58 val_58 +64 val_64 64 val_64 +66 val_66 66 val_66 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +70 val_70 70 val_70 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +72 val_72 72 val_72 +74 val_74 74 val_74 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +76 val_76 76 val_76 +78 val_78 78 val_78 +80 val_80 80 val_80 +82 val_82 82 val_82 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +84 val_84 84 val_84 +86 val_86 86 val_86 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +90 val_90 90 val_90 +92 val_92 92 val_92 +96 val_96 96 val_96 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +98 val_98 98 val_98 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +100 val_100 100 val_100 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +104 val_104 104 val_104 +114 val_114 114 val_114 +116 val_116 116 val_116 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +118 val_118 118 val_118 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +120 val_120 120 val_120 +126 val_126 126 val_126 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +128 val_128 128 val_128 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +134 val_134 134 val_134 +136 val_136 136 val_136 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +138 val_138 138 val_138 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +152 val_152 152 val_152 +156 val_156 156 val_156 +158 val_158 158 val_158 +160 val_160 160 val_160 +162 val_162 162 val_162 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +164 val_164 164 val_164 +166 val_166 166 val_166 +168 val_168 168 val_168 +170 val_170 170 val_170 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +172 val_172 172 val_172 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +174 val_174 174 val_174 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +176 val_176 176 val_176 +178 val_178 178 val_178 +180 val_180 180 val_180 +186 val_186 186 val_186 +190 val_190 190 val_190 +192 val_192 192 val_192 +194 val_194 194 val_194 +196 val_196 196 val_196 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +200 val_200 200 val_200 +202 val_202 202 val_202 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +208 val_208 208 val_208 +214 val_214 214 val_214 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +216 val_216 216 val_216 +218 val_218 218 val_218 +222 val_222 222 val_222 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +224 val_224 224 val_224 +226 val_226 226 val_226 +228 val_228 228 val_228 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +230 val_230 230 val_230 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +238 val_238 238 val_238 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +242 val_242 242 val_242 +244 val_244 244 val_244 +248 val_248 248 val_248 +252 val_252 252 val_252 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +256 val_256 256 val_256 +258 val_258 258 val_258 +260 val_260 260 val_260 +262 val_262 262 val_262 +266 val_266 266 val_266 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +272 val_272 272 val_272 +274 val_274 274 val_274 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +278 val_278 278 val_278 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +280 val_280 280 val_280 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +282 val_282 282 val_282 +284 val_284 284 val_284 +286 val_286 286 val_286 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +288 val_288 288 val_288 +292 val_292 292 val_292 +296 val_296 296 val_296 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +298 val_298 298 val_298 +302 val_302 302 val_302 +306 val_306 306 val_306 +308 val_308 308 val_308 +310 val_310 310 val_310 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +316 val_316 316 val_316 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +318 val_318 318 val_318 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +322 val_322 322 val_322 +332 val_332 332 val_332 +336 val_336 336 val_336 +338 val_338 338 val_338 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +342 val_342 342 val_342 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +344 val_344 344 val_344 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +348 val_348 348 val_348 +356 val_356 356 val_356 +360 val_360 360 val_360 +362 val_362 362 val_362 +364 val_364 364 val_364 +366 val_366 366 val_366 +368 val_368 368 val_368 +374 val_374 374 val_374 +378 val_378 378 val_378 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +382 val_382 382 val_382 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +384 val_384 384 val_384 +386 val_386 386 val_386 +392 val_392 392 val_392 +394 val_394 394 val_394 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +396 val_396 396 val_396 +400 val_400 400 val_400 +402 val_402 402 val_402 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +404 val_404 404 val_404 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +414 val_414 414 val_414 +418 val_418 418 val_418 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +424 val_424 424 val_424 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +430 val_430 430 val_430 +432 val_432 432 val_432 +436 val_436 436 val_436 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +438 val_438 438 val_438 +444 val_444 444 val_444 +446 val_446 446 val_446 +448 val_448 448 val_448 +452 val_452 452 val_452 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +454 val_454 454 val_454 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +458 val_458 458 val_458 +460 val_460 460 val_460 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +462 val_462 462 val_462 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +466 val_466 466 val_466 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +468 val_468 468 val_468 +470 val_470 470 val_470 +472 val_472 472 val_472 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +478 val_478 478 val_478 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +480 val_480 480 val_480 +482 val_482 482 val_482 +484 val_484 484 val_484 +490 val_490 490 val_490 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +492 val_492 492 val_492 +494 val_494 494 val_494 +496 val_496 496 val_496 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +498 val_498 498 val_498 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +5 val_5 5 val_5 +9 val_9 9 val_9 +11 val_11 11 val_11 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +15 val_15 15 val_15 +17 val_17 17 val_17 +19 val_19 19 val_19 +27 val_27 27 val_27 +33 val_33 33 val_33 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +35 val_35 35 val_35 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +37 val_37 37 val_37 +41 val_41 41 val_41 +43 val_43 43 val_43 +47 val_47 47 val_47 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +51 val_51 51 val_51 +53 val_53 53 val_53 +57 val_57 57 val_57 +65 val_65 65 val_65 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +67 val_67 67 val_67 +69 val_69 69 val_69 +77 val_77 77 val_77 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +83 val_83 83 val_83 +85 val_85 85 val_85 +87 val_87 87 val_87 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +95 val_95 95 val_95 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +97 val_97 97 val_97 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +103 val_103 103 val_103 +105 val_105 105 val_105 +111 val_111 111 val_111 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +113 val_113 113 val_113 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +119 val_119 119 val_119 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +125 val_125 125 val_125 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +129 val_129 129 val_129 +131 val_131 131 val_131 +133 val_133 133 val_133 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +137 val_137 137 val_137 +143 val_143 143 val_143 +145 val_145 145 val_145 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +149 val_149 149 val_149 +153 val_153 153 val_153 +155 val_155 155 val_155 +157 val_157 157 val_157 +163 val_163 163 val_163 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +165 val_165 165 val_165 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +167 val_167 167 val_167 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +169 val_169 169 val_169 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +175 val_175 175 val_175 +177 val_177 177 val_177 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +179 val_179 179 val_179 +181 val_181 181 val_181 +183 val_183 183 val_183 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +187 val_187 187 val_187 +189 val_189 189 val_189 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +191 val_191 191 val_191 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +193 val_193 193 val_193 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +195 val_195 195 val_195 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +197 val_197 197 val_197 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +199 val_199 199 val_199 +201 val_201 201 val_201 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +203 val_203 203 val_203 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +205 val_205 205 val_205 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +207 val_207 207 val_207 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +209 val_209 209 val_209 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +213 val_213 213 val_213 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +217 val_217 217 val_217 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +219 val_219 219 val_219 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +221 val_221 221 val_221 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +223 val_223 223 val_223 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +229 val_229 229 val_229 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +233 val_233 233 val_233 +235 val_235 235 val_235 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +237 val_237 237 val_237 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +239 val_239 239 val_239 +241 val_241 241 val_241 +247 val_247 247 val_247 +249 val_249 249 val_249 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +255 val_255 255 val_255 +257 val_257 257 val_257 +263 val_263 263 val_263 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +265 val_265 265 val_265 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +275 val_275 275 val_275 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +277 val_277 277 val_277 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +281 val_281 281 val_281 +283 val_283 283 val_283 +285 val_285 285 val_285 +287 val_287 287 val_287 +289 val_289 289 val_289 +291 val_291 291 val_291 +305 val_305 305 val_305 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +307 val_307 307 val_307 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +309 val_309 309 val_309 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +315 val_315 315 val_315 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +317 val_317 317 val_317 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +321 val_321 321 val_321 +323 val_323 323 val_323 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +325 val_325 325 val_325 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +327 val_327 327 val_327 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +331 val_331 331 val_331 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +333 val_333 333 val_333 +335 val_335 335 val_335 +339 val_339 339 val_339 +341 val_341 341 val_341 +345 val_345 345 val_345 +351 val_351 351 val_351 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +353 val_353 353 val_353 +365 val_365 365 val_365 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +367 val_367 367 val_367 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +369 val_369 369 val_369 +373 val_373 373 val_373 +375 val_375 375 val_375 +377 val_377 377 val_377 +379 val_379 379 val_379 +389 val_389 389 val_389 +393 val_393 393 val_393 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +395 val_395 395 val_395 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +397 val_397 397 val_397 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +399 val_399 399 val_399 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +401 val_401 401 val_401 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +403 val_403 403 val_403 +407 val_407 407 val_407 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +409 val_409 409 val_409 +411 val_411 411 val_411 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +413 val_413 413 val_413 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +417 val_417 417 val_417 +419 val_419 419 val_419 +421 val_421 421 val_421 +427 val_427 427 val_427 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +429 val_429 429 val_429 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +431 val_431 431 val_431 +435 val_435 435 val_435 +437 val_437 437 val_437 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +439 val_439 439 val_439 +443 val_443 443 val_443 +449 val_449 449 val_449 +453 val_453 453 val_453 +455 val_455 455 val_455 +457 val_457 457 val_457 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +459 val_459 459 val_459 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +463 val_463 463 val_463 +467 val_467 467 val_467 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +469 val_469 469 val_469 +475 val_475 475 val_475 +477 val_477 477 val_477 +479 val_479 479 val_479 +481 val_481 481 val_481 +483 val_483 483 val_483 +485 val_485 485 val_485 +487 val_487 487 val_487 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +489 val_489 489 val_489 +491 val_491 491 val_491 +493 val_493 493 val_493 +495 val_495 495 val_495 +PREHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket4_1 a) (TOK_TABREF smb_bucket4_2 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_ALLCOLREF)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL a) key) 1000)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b + TableScan + alias: b + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Filter Operator + predicate: + expr: (_col0 > 1000) + type: boolean + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-35_626_8108094048335552298/10000 +POSTHOOK: query: select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-35_626_8108094048335552298/10000 +PREHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF smb_bucket4_1 a) (TOK_TABREF smb_bucket4_2 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 b))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL a) key) 1000)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: + expr: (key > 1000) + type: boolean + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Filter Operator + predicate: + expr: (_col0 > 1000) + type: boolean + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-40_809_6155812110614982847/10000 +POSTHOOK: query: select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-40_809_6155812110614982847/10000 +PREHOOK: query: explain +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key join smb_bucket4_2 c on b.key = c.key where a.key>1000 +PREHOOK: type: QUERY +POSTHOOK: query: explain +select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key join smb_bucket4_2 c on b.key = c.key where a.key>1000 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF smb_bucket4_1 a) (TOK_TABREF smb_bucket4_2 b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF smb_bucket4_2 c) (= (. (TOK_TABLE_OR_COL b) 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 b c))) (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL a) key) 1000)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: + expr: (key > 1000) + type: boolean + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {key} {value} + 1 {key} {value} + 2 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Filter Operator + predicate: + expr: (_col0 > 1000) + type: boolean + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key join smb_bucket4_2 c on b.key = c.key where a.key>1000 +PREHOOK: type: QUERY +PREHOOK: Input: default@smb_bucket4_2 +PREHOOK: Input: default@smb_bucket4_1 +PREHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-46_064_6690209903612725310/10000 +POSTHOOK: query: select /*+mapjoin(b,c)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key join smb_bucket4_2 c on b.key = c.key where a.key>1000 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@smb_bucket4_2 +POSTHOOK: Input: default@smb_bucket4_1 +POSTHOOK: Output: file:/Users/heyongqiang/Documents/workspace/Hive-Test/build/ql/scratchdir/hive_2010-03-03_15-59-46_064_6690209903612725310/10000 +PREHOOK: query: drop table smb_bucket4_1 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket4_1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket4_1 +PREHOOK: query: drop table smb_bucket4_2 +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table smb_bucket4_2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Output: default@smb_bucket4_2