Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java =================================================================== --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -569,6 +569,7 @@ // It creates sub-directories in the final output, so should not be turned on in systems // where MAPREDUCE-1501 is not present HIVE_OPTIMIZE_UNION_REMOVE("hive.optimize.union.remove", false), + HIVEOPTCORRELATION("hive.optimize.correlation", false), // exploit intra-query correlations // whether hadoop map-reduce supports sub-directories. It was added by MAPREDUCE-1501. // Some optimizations can only be performed if the version of hadoop being used supports Index: conf/hive-default.xml.template =================================================================== --- conf/hive-default.xml.template +++ conf/hive-default.xml.template @@ -1121,6 +1121,12 @@ + hive.optimize.correlation + false + exploit intra-query correlations. + + + hive.optimize.reducededuplication.min.reducer 4 Reduce deduplication merges two RSs by moving key/parts/reducer-num of the child RS to parent RS. Index: ql/if/queryplan.thrift =================================================================== --- ql/if/queryplan.thrift +++ ql/if/queryplan.thrift @@ -54,6 +54,8 @@ HASHTABLESINK, HASHTABLEDUMMY, PTF, + MUX, + DEMUX, } struct Operator { Index: ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java =================================================================== --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java @@ -30,7 +30,9 @@ LATERALVIEWFORWARD(15), HASHTABLESINK(16), HASHTABLEDUMMY(17), - PTF(18); + PTF(18), + MUX(19), + DEMUX(20); private final int value; @@ -49,7 +51,7 @@ * Find a the enum type by its integer value, as defined in the Thrift IDL. * @return null if the value is not found. */ - public static OperatorType findByValue(int value) { + public static OperatorType findByValue(int value) { switch (value) { case 0: return JOIN; @@ -89,6 +91,10 @@ return HASHTABLEDUMMY; case 18: return PTF; + case 19: + return MUX; + case 20: + return DEMUX; default: return null; } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java @@ -331,6 +331,7 @@ for (AbstractRowContainer> alw : storage) { alw.clear(); } + super.startGroup(); } protected int getNextSize(int sz) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/DemuxOperator.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/DemuxOperator.java @@ -0,0 +1,392 @@ +/** + * 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.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.DemuxDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * DemuxOperator is an operator used by MapReduce Jobs optimized by + * CorrelationOptimizer. If used, DemuxOperator is the first operator in reduce + * phase. In the case that multiple operation paths are merged into a single one, it will dispatch + * the record to corresponding child operators (Join or GBY). + */ +public class DemuxOperator extends Operator + implements Serializable { + + private static final long serialVersionUID = 1L; + protected static final Log LOG = LogFactory.getLog(DemuxOperator.class.getName()); + + /** + * Handler is used to assign original tag (oldTag) to a row and + * track how many rows are forwarded to every child of DemuxOperator. + */ + protected static class Handler { + // oldTag is the tag assigned to ReduceSinkOperators BEFORE Correlation Optimizer + // optimizes the operator tree. newTag is the tag assigned to ReduceSinkOperators + // AFTER Correlation Optimizer optimizes the operator tree. + // Example: we have an operator tree shown below ... + // JOIN2 + // / \ + // GBY1 JOIN1 + // | / \ + // RS1 RS2 RS3 + // If GBY1, JOIN1, and JOIN2 are executed in the same Reducer + // (optimized by Correlation Optimizer), we will have ... + // oldTag: RS1:0, RS2:0, RS3:1 + // newTag: RS1:0, RS2:1, RS3:2 + // We need to know the mapping from the newTag to oldTag and revert + // the newTag to oldTag to make operators in the operator tree + // function correctly. + private final byte newTag; + private final byte oldTag; + private final byte childIndex; + private final ByteWritable oldTagByteWritable; + private final List forwardedRow; + + // counters for debugging + private transient long cntr = 0; + private transient long nextCntr = 1; + + private long getNextCntr(long cntr) { + // A very simple counter to keep track of number of rows processed by an + // operator. It dumps + // every 1 million times, and quickly before that + if (cntr >= 1000000) { + return cntr + 1000000; + } + return 10 * cntr; + } + + public long getCntr() { + return this.cntr; + } + + private final Log log; + private final boolean isLogInfoEnabled; + private final String id; + + public Handler(byte newTag, byte childIndex, byte oldTag, Log LOG, String id) + throws HiveException { + this.newTag = newTag; + this.oldTag = oldTag; + this.childIndex = childIndex; + this.oldTagByteWritable = new ByteWritable(oldTag); + this.log = LOG; + this.isLogInfoEnabled = LOG.isInfoEnabled(); + this.id = id; + this.forwardedRow = new ArrayList(3); + } + + public byte getOldTag() { + return oldTag; + } + + public Object process(Object row) throws HiveException { + forwardedRow.clear(); + List thisRow = (List) row; + forwardedRow.add(thisRow.get(0)); + forwardedRow.add(thisRow.get(1)); + forwardedRow.add(oldTagByteWritable); + + if (isLogInfoEnabled) { + cntr++; + if (cntr == nextCntr) { + log.info(id + " (newTag, childIndex, oldTag)=(" + newTag + ", " + childIndex + ", " + + oldTag + "), forwarding " + cntr + " rows"); + nextCntr = getNextCntr(cntr); + } + } + + return forwardedRow; + } + + public void printCloseOpLog() { + log.info(id + " (newTag, childIndex, oldTag)=(" + newTag + ", " + childIndex + ", " + + oldTag + "), forwarded " + cntr + " rows"); + } + } + + // The mapping from a newTag to its corresponding oldTag. Please see comments in + // DemuxOperator.Handler for explanations of newTag and oldTag. + private Map newTagToOldTag = + new HashMap(); + + // The mapping from a newTag to the index of the corresponding child + // of this operator. + private Map newTagToChildIndex = + new HashMap(); + + // The mapping from a newTag to its corresponding handler + private Map newTagToDispatchHandler = + new HashMap(); + + // The mapping from the index of a child operator to its corresponding + // inputObjectInspectors + private Map childInputObjInspectors; + + private int childrenDone; + + // Since DemuxOperator may appear multiple times in MuxOperator's parents list. + // We use newChildIndexTag instead of childOperatorsTag. + // Example: + // JOIN + // | + // MUX + // / | \ + // / | \ + // / | \ + // | GBY | + // \ | / + // \ | / + // DEMUX + // In this case, the parent list of MUX is [DEMUX, GBY, DEMUX], + // so we need to have two childOperatorsTags (the index of this DemuxOperator in + // its children's parents lists, also see childOperatorsTag in Operator) at here. + private List> newChildOperatorsTag; + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + this.newTagToOldTag = conf.getNewTagToOldTag(); + this.newTagToChildIndex = conf.getNewTagToChildIndex(); + this.newTagToDispatchHandler = new HashMap(); + this.childInputObjInspectors = new HashMap(); + + // For every newTag (every newTag corresponds to a ReduceSinkOperator), + // create a handler. Also, we initialize childInputObjInspectors at here. + for (Entry entry: newTagToOldTag.entrySet()) { + int newTag = entry.getKey(); + int oldTag = entry.getValue(); + int childIndex = newTagToChildIndex.get(newTag); + Handler handler = + new Handler((byte)newTag, (byte)childIndex, (byte)oldTag, LOG, id); + newTagToDispatchHandler.put(newTag, handler); + int childParentsCount = conf.getChildIndexToOriginalNumParents().get(childIndex); + childInputObjInspectors.put(childIndex, new ObjectInspector[childParentsCount]); + } + + try { + // We populate inputInspectors for all children of this DemuxOperator. + // Those inputObjectInspectors are stored in childInputObjInspectors. + for (Entry e1: newTagToOldTag.entrySet()) { + int newTag = e1.getKey(); + int oldTag = e1.getValue(); + int childIndex = newTagToChildIndex.get(newTag); + TableDesc keyTableDesc = conf.getKeysSerializeInfos().get(newTag); + Deserializer inputKeyDeserializer = (SerDe) ReflectionUtils.newInstance(keyTableDesc + .getDeserializerClass(), null); + inputKeyDeserializer.initialize(null, keyTableDesc.getProperties()); + + TableDesc valueTableDesc = conf.getValuesSerializeInfos().get(newTag); + Deserializer inputValueDeserializer = (SerDe) ReflectionUtils.newInstance(valueTableDesc + .getDeserializerClass(), null); + inputValueDeserializer.initialize(null, valueTableDesc.getProperties()); + + List oi = new ArrayList(); + oi.add(inputKeyDeserializer.getObjectInspector()); + oi.add(inputValueDeserializer.getObjectInspector()); + oi.add(PrimitiveObjectInspectorFactory.writableByteObjectInspector); + ObjectInspector[] ois = childInputObjInspectors.get(childIndex); + ois[oldTag] = ObjectInspectorFactory + .getStandardStructObjectInspector(Utilities.fieldNameList, oi); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + this.childrenDone = 0; + newChildOperatorsTag = new ArrayList>(); + for (Operator child: childOperators) { + List childOperatorTags = new ArrayList(); + if (child instanceof MuxOperator) { + // This DemuxOperator can appear multiple times in MuxOperator's + // parentOperators + int index = 0; + for (Operator parent: child.getParentOperators()) { + if (this == parent) { + childOperatorTags.add(index); + } + index++; + } + } else { + childOperatorTags.add(child.getParentOperators().indexOf(this)); + } + newChildOperatorsTag.add(childOperatorTags); + } + LOG.info("newChildOperatorsTag " + newChildOperatorsTag); + initializeChildren(hconf); + } + + // Each child should has its own outputObjInspector + @Override + protected void initializeChildren(Configuration hconf) throws HiveException { + state = State.INIT; + LOG.info("Operator " + id + " " + getName() + " initialized"); + if (childOperators == null) { + return; + } + LOG.info("Initializing children of " + id + " " + getName()); + for (int i = 0; i < childOperatorsArray.length; i++) { + LOG.info("Initializing child " + i + " " + childOperatorsArray[i].getIdentifier() + " " + + childOperatorsArray[i].getName() + + " " + childInputObjInspectors.get(i).length); + // We need to initialize those MuxOperators first because if we first + // initialize other operators, the states of all parents of those MuxOperators + // are INIT (including this DemuxOperator), + // but the inputInspector of those MuxOperators has not been set. + if (childOperatorsArray[i] instanceof MuxOperator) { + // If this DemuxOperator directly connects to a MuxOperator, + // that MuxOperator must be the parent of a JoinOperator. + // In this case, that MuxOperator should be initialized + // by multiple parents (of that MuxOperator). + ObjectInspector[] ois = childInputObjInspectors.get(i); + for (int j = 0; j < ois.length; j++) { + if (ois[j] != null) { + childOperatorsArray[i].initialize(hconf, ois[j], j); + } + } + } else { + continue; + } + if (reporter != null) { + childOperatorsArray[i].setReporter(reporter); + } + } + for (int i = 0; i < childOperatorsArray.length; i++) { + LOG.info("Initializing child " + i + " " + childOperatorsArray[i].getIdentifier() + " " + + childOperatorsArray[i].getName() + + " " + childInputObjInspectors.get(i).length); + if (!(childOperatorsArray[i] instanceof MuxOperator)) { + childOperatorsArray[i].initialize(hconf, childInputObjInspectors.get(i)); + } else { + continue; + } + if (reporter != null) { + childOperatorsArray[i].setReporter(reporter); + } + } + } + + @Override + public void processOp(Object row, int tag) throws HiveException { + int newTag = tag; + forward(row, inputObjInspectors[newTag]); + } + + @Override + public void forward(Object row, ObjectInspector rowInspector) + throws HiveException { + if ((++outputRows % 1000) == 0) { + if (counterNameToEnum != null) { + incrCounter(numOutputRowsCntr, outputRows); + outputRows = 0; + } + } + + if (childOperatorsArray == null && childOperators != null) { + throw new HiveException("Internal Hive error during operator initialization."); + } + + if ((childOperatorsArray == null) || (getDone())) { + return; + } + + List thisRow = (List) row; + assert thisRow.size() == 3; + int newTag = ((ByteWritable) thisRow.get(2)).get(); + Handler handler = newTagToDispatchHandler.get(newTag); + int childIndex = newTagToChildIndex.get(newTag); + Operator o = childOperatorsArray[childIndex]; + if (o.getDone()) { + childrenDone++; + } else { + o.process(handler.process(row), handler.getOldTag()); + } + + // if all children are done, this operator is also done + if (childrenDone == childOperatorsArray.length) { + setDone(true); + } + + } + + @Override + protected void closeOp(boolean abort) throws HiveException { + // log the number of rows forwarded from each dispatcherHandler + for (Handler handler: newTagToDispatchHandler.values()) { + handler.printCloseOpLog(); + } + } + + @Override + public void endGroup() throws HiveException { + if (childOperators == null) { + return; + } + + if (fatalError) { + return; + } + + for (int i = 0; i < childOperatorsArray.length; i++) { + Operator child = childOperatorsArray[i]; + child.flush(); + child.endGroup(); + for (Integer childTag: newChildOperatorsTag.get(i)) { + child.processGroup(childTag); + } + } + } + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return getOperatorName(); + } + + static public String getOperatorName() { + return "DEMUX"; + } + + @Override + public OperatorType getType() { + return OperatorType.DEMUX; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java @@ -707,6 +707,7 @@ @Override public void startGroup() throws HiveException { firstRowInGroup = true; + super.startGroup(); } @Override @@ -750,7 +751,7 @@ + " #total = " + numRowsInput + " reduction = " + 1.0 * (numRowsHashTbl / numRowsInput) + " minReduction = " + minReductionHashAggr); - flush(true); + flushHashTable(true); hashAggr = false; } else { LOG.trace("Hash Aggr Enabled: #hash table = " + numRowsHashTbl @@ -835,7 +836,7 @@ // happen at boundaries if ((!groupKeyIsNotReduceKey || firstRowInGroup) && shouldBeFlushed(newKeys)) { - flush(false); + flushHashTable(false); } } @@ -983,7 +984,12 @@ return length; } - private void flush(boolean complete) throws HiveException { + /** + * Flush hash table. This method is used by hash-based aggregations + * @param complete + * @throws HiveException + */ + private void flushHashTable(boolean complete) throws HiveException { countAfterReport = 0; @@ -1048,6 +1054,42 @@ } /** + * Forward all aggregations to children. It is only used by DemuxOperator. + * @throws HiveException + */ + @Override + public void flush() throws HiveException{ + try { + if (hashAggregations != null) { + LOG.info("Begin Hash Table flush: size = " + + hashAggregations.size()); + Iterator iter = hashAggregations.entrySet().iterator(); + while (iter.hasNext()) { + Map.Entry m = (Map.Entry) iter + .next(); + + forward(m.getKey().getKeyArray(), m.getValue()); + iter.remove(); + } + hashAggregations.clear(); + } else if (aggregations != null) { + // sort-based aggregations + if (currentKeys != null) { + forward(currentKeys.getKeyArray(), aggregations); + } + currentKeys = null; + } else { + // The GroupByOperator is not initialized, which means there is no + // data + // (since we initialize the operators when we see the first record). + // Just do nothing here. + } + } catch (Exception e) { + throw new HiveException(e); + } + } + + /** * We need to forward all the aggregations to children. * */ @@ -1088,33 +1130,9 @@ // create dummy keys - size 0 forward(new Object[0], aggregations); } else { - if (hashAggregations != null) { - LOG.info("Begin Hash Table flush at close: size = " - + hashAggregations.size()); - Iterator iter = hashAggregations.entrySet().iterator(); - while (iter.hasNext()) { - Map.Entry m = (Map.Entry) iter - .next(); - - forward(m.getKey().getKeyArray(), m.getValue()); - iter.remove(); - } - hashAggregations.clear(); - } else if (aggregations != null) { - // sort-based aggregations - if (currentKeys != null) { - forward(currentKeys.getKeyArray(), aggregations); - } - currentKeys = null; - } else { - // The GroupByOperator is not initialized, which means there is no - // data - // (since we initialize the operators when we see the first record). - // Just do nothing here. - } + flush(); } } catch (Exception e) { - e.printStackTrace(); throw new HiveException(e); } } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MuxOperator.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MuxOperator.java @@ -0,0 +1,343 @@ +/** + * 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.Arrays; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.MuxDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; + +/** + * MuxOperator is used in the Reduce side of MapReduce jobs optimized by Correlation Optimizer. + * Correlation Optimizer will remove unnecessary ReduceSinkOperaotrs, + * and MuxOperators are used to replace those ReduceSinkOperaotrs. + * Example: The original operator tree is ... + * JOIN2 + * / \ + * RS4 RS5 + * / \ + * GBY1 JOIN1 + * | / \ + * RS1 RS2 RS3 + * If GBY1, JOIN1, and JOIN2 can be executed in the same reducer + * (optimized by Correlation Optimizer). + * The new operator tree will be ... + * JOIN2 + * | + * MUX + * / \ + * GBY1 JOIN1 + * \ / + * DEMUX + * / | \ + * / | \ + * / | \ + * RS1 RS2 RS3 + * + * A MuxOperator has two functions. + * First, it will construct key, value and tag structure for + * the input of Join Operators. + * Second, it is a part of operator coordination mechanism which makes sure the operator tree + * in the Reducer can work correctly. + */ +public class MuxOperator extends Operator implements Serializable{ + + private static final long serialVersionUID = 1L; + protected static final Log LOG = LogFactory.getLog(MuxOperator.class.getName()); + + /** + * Handler is used to construct key-value-tag structure and assign original tag to a row. + */ + protected static class Handler { + private final ObjectInspector outputObjInspector; + private final int tag; + private final ByteWritable tagByteWritable; + /** + * The evaluators for the key columns. Key columns decide the sort order on + * the reducer side. Key columns are passed to the reducer in the "key". + */ + private final ExprNodeEvaluator[] keyEval; + /** + * The evaluators for the value columns. Value columns are passed to reducer + * in the "value". + */ + private final ExprNodeEvaluator[] valueEval; + private final Object[] outputKey; + private final Object[] outputValue; + private final List forwardedRow; + + public Handler(ObjectInspector inputObjInspector, + List keyCols, + List valueCols, + List outputKeyColumnNames, + List outputValueColumnNames, + Integer tag) throws HiveException { + + keyEval = new ExprNodeEvaluator[keyCols.size()]; + int i = 0; + for (ExprNodeDesc e: keyCols) { + keyEval[i++] = ExprNodeEvaluatorFactory.get(e); + } + outputKey = new Object[keyEval.length]; + + valueEval = new ExprNodeEvaluator[valueCols.size()]; + i = 0; + for (ExprNodeDesc e: valueCols) { + valueEval[i++] = ExprNodeEvaluatorFactory.get(e); + } + outputValue = new Object[valueEval.length]; + + this.tag = tag; + this.tagByteWritable = new ByteWritable((byte)tag.intValue()); + + ObjectInspector keyObjectInspector = initEvaluatorsAndReturnStruct(keyEval, + outputKeyColumnNames, inputObjInspector); + ObjectInspector valueObjectInspector = initEvaluatorsAndReturnStruct(valueEval, + outputValueColumnNames, inputObjInspector); + List ois = new ArrayList(); + ois.add(keyObjectInspector); + ois.add(valueObjectInspector); + ois.add(PrimitiveObjectInspectorFactory.writableByteObjectInspector); + this.outputObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector( + Utilities.fieldNameList, ois); + this.forwardedRow = new ArrayList(3); + } + + public ObjectInspector getOutputObjInspector() { + return outputObjInspector; + } + + public int getTag() { + return tag; + } + + public Object process(Object row) throws HiveException { + // Evaluate the keys + for (int i = 0; i < keyEval.length; i++) { + outputKey[i] = keyEval[i].evaluate(row); + } + // Evaluate the value + for (int i = 0; i < valueEval.length; i++) { + outputValue[i] = valueEval[i].evaluate(row); + } + forwardedRow.clear(); + // JoinOperator assumes the key is backed by an list. + // To be consistent, the value array is also converted + // to a list. + forwardedRow.add(Arrays.asList(outputKey)); + forwardedRow.add(Arrays.asList(outputValue)); + forwardedRow.add(tagByteWritable); + return forwardedRow; + } + } + + private transient ObjectInspector[] outputObjectInspectors; + private transient int numParents; + private transient boolean[] forward; + private transient boolean[] processGroupCalled; + private Handler[] handlers; + + //counters for debugging + private transient long[] cntr; + private transient long[] nextCntr; + + private long getNextCntr(long cntr) { + // A very simple counter to keep track of number of rows processed by an + // operator. It dumps + // every 1 million times, and quickly before that + if (cntr >= 1000000) { + return cntr + 1000000; + } + return 10 * cntr; + } + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + // A MuxOperator should only has a single child + if (childOperatorsArray.length != 1) { + throw new HiveException( + "Expected number of children is 1. Found : " + childOperatorsArray.length); + } + numParents = getNumParent(); + forward = new boolean[numParents]; + processGroupCalled = new boolean[numParents]; + outputObjectInspectors = new ObjectInspector[numParents]; + handlers = new Handler[numParents]; + cntr = new long[numParents]; + nextCntr = new long[numParents]; + for (int i = 0; i < numParents; i++) { + processGroupCalled[i] = false; + if (conf.getParentToKeyCols().get(i) == null) { + // We do not need to evaluate the input row for this parent. + // So, we can just forward it to the child of this MuxOperator. + handlers[i] = null; + forward[i] = true; + outputObjectInspectors[i] = inputObjInspectors[i]; + } else { + handlers[i] = new Handler( + inputObjInspectors[i], + conf.getParentToKeyCols().get(i), + conf.getParentToValueCols().get(i), + conf.getParentToOutputKeyColumnNames().get(i), + conf.getParentToOutputValueColumnNames().get(i), + conf.getParentToTag().get(i)); + forward[i] = false; + outputObjectInspectors[i] = handlers[i].getOutputObjInspector(); + } + cntr[i] = 0; + nextCntr[i] = 1; + } + initializeChildren(hconf); + } + + /** + * Calls initialize on each of the children with outputObjetInspector as the + * output row format. + */ + @Override + protected void initializeChildren(Configuration hconf) throws HiveException { + state = State.INIT; + LOG.info("Operator " + id + " " + getName() + " initialized"); + if (childOperators == null) { + return; + } + LOG.info("Initializing children of " + id + " " + getName()); + childOperatorsArray[0].initialize(hconf, outputObjectInspectors); + if (reporter != null) { + childOperatorsArray[0].setReporter(reporter); + } + } + + @Override + public void processOp(Object row, int tag) throws HiveException { + forward(row, tag); + } + + protected void forward(Object row, int tag) + throws HiveException { + + if (childOperatorsArray == null && childOperators != null) { + throw new HiveException( + "Internal Hive error during operator initialization."); + } + + if ((childOperatorsArray == null) || (getDone())) { + return; + } + + int childrenDone = 0; + for (int i = 0; i < childOperatorsArray.length; i++) { + Operator o = childOperatorsArray[i]; + if (o.getDone()) { + childrenDone++; + } else { + if (forward[tag]) { + // No need to evaluate, just forward it. + o.process(row, tag); + } else { + // Call the corresponding handler to evaluate this row and + // forward the result + o.process(handlers[tag].process(row), handlers[tag].getTag()); + } + } + } + + if (isLogInfoEnabled) { + cntr[tag]++; + if (cntr[tag] == nextCntr[tag]) { + LOG.info(id + ", tag=" + tag + ", forwarding " + cntr[tag] + " rows"); + nextCntr[tag] = getNextCntr(cntr[tag]); + } + } + + // if all children are done, this operator is also done + if (childrenDone == childOperatorsArray.length) { + setDone(true); + } + } + + @Override + public void startGroup() throws HiveException{ + for (int i = 0; i < numParents; i++) { + processGroupCalled[i] = false; + } + super.startGroup(); + } + + @Override + public void endGroup() throws HiveException { + // do nothing + } + + @Override + public void processGroup(int tag) throws HiveException { + processGroupCalled[tag] = true; + boolean shouldProceed = true; + for (int i = 0; i < numParents; i++) { + if (!processGroupCalled[i]) { + shouldProceed = false; + break; + } + } + if (shouldProceed) { + Operator child = childOperatorsArray[0]; + int childTag = childOperatorsTag[0]; + child.flush(); + child.endGroup(); + child.processGroup(childTag); + } + } + + @Override + protected void closeOp(boolean abort) throws HiveException { + for (int i = 0; i < numParents; i++) { + LOG.info(id + ", tag=" + i + ", forwarded " + cntr[i] + " rows"); + } + } + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return getOperatorName(); + } + + static public String getOperatorName() { + return "MUX"; + } + + @Override + public OperatorType getType() { + return OperatorType.MUX; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java @@ -436,7 +436,7 @@ * parent operator id * @throws HiveException */ - private void initialize(Configuration hconf, ObjectInspector inputOI, + protected void initialize(Configuration hconf, ObjectInspector inputOI, int parentId) throws HiveException { LOG.info("Initializing child " + id + " " + getName()); // Double the size of the array if needed @@ -524,7 +524,7 @@ LOG.debug("Start group Done"); } - // If a operator wants to do some work at the end of a group + // If an operator wants to do some work at the end of a group public void endGroup() throws HiveException { LOG.debug("Ending group"); @@ -544,6 +544,20 @@ LOG.debug("End group Done"); } + // an blocking operator (e.g. GroupByOperator and JoinOperator) can + // override this method to forward its outputs + public void flush() throws HiveException { + } + + public void processGroup(int tag) throws HiveException { + if (childOperators == null) { + return; + } + for (int i = 0; i < childOperatorsArray.length; i++) { + childOperatorsArray[i].processGroup(childOperatorsTag[i]); + } + } + protected boolean allInitializedParentsAreClosed() { if (parentOperators != null) { for (Operator parent : parentOperators) { @@ -1481,6 +1495,7 @@ return true; } + @Override public String toString() { return getName() + "[" + getIdentifier() + "]"; } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java @@ -23,6 +23,8 @@ import java.util.Map; import org.apache.hadoop.hive.ql.plan.CollectDesc; +import org.apache.hadoop.hive.ql.plan.MuxDesc; +import org.apache.hadoop.hive.ql.plan.DemuxDesc; import org.apache.hadoop.hive.ql.plan.DummyStoreDesc; import org.apache.hadoop.hive.ql.plan.ExtractDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; @@ -98,6 +100,10 @@ HashTableSinkOperator.class)); opvec.add(new OpTuple(DummyStoreDesc.class, DummyStoreOperator.class)); + opvec.add(new OpTuple(DemuxDesc.class, + DemuxOperator.class)); + opvec.add(new OpTuple(MuxDesc.class, + MuxOperator.class)); } public static Operator get(Class opClass) { @@ -257,7 +263,7 @@ public static Operator getAndMakeChild(T conf, RowSchema rwsch, Map colExprMap, Operator... oplist) { Operator ret = getAndMakeChild(conf, rwsch, oplist); - ret.setColumnExprMap(colExprMap); + ret.setColumnExprMap(colExprMap); return (ret); } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -168,13 +168,23 @@ public static String HADOOP_LOCAL_FS = "file:///"; /** - * ReduceField. - * + * ReduceField: + * KEY: record key + * VALUE: record value + * ALIAS: the tag identifying the source of a record */ public static enum ReduceField { KEY, VALUE, ALIAS }; + public static List fieldNameList; + static { + fieldNameList = new ArrayList(); + for (ReduceField r : ReduceField.values()) { + fieldNameList.add(r.toString()); + } + } + private Utilities() { // prevent instantiation } @@ -236,15 +246,18 @@ public static void setWorkflowAdjacencies(Configuration conf, QueryPlan plan) { try { Graph stageGraph = plan.getQueryPlan().getStageGraph(); - if (stageGraph == null) + if (stageGraph == null) { return; + } List adjList = stageGraph.getAdjacencyList(); - if (adjList == null) + if (adjList == null) { return; + } for (Adjacency adj : adjList) { List children = adj.getChildren(); - if (children == null || children.isEmpty()) + if (children == null || children.isEmpty()) { return; + } conf.setStrings("mapreduce.workflow.adjacency."+adj.getNode(), children.toArray(new String[children.size()])); } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; +import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -74,7 +75,6 @@ private long cntr = 0; private long nextCntr = 1; - private static String[] fieldNames; public static final Log l4j = LogFactory.getLog("ExecReducer"); private boolean isLogInfoEnabled = false; @@ -86,13 +86,6 @@ // Input value serde needs to be an array to support different SerDe // for different tags private final SerDe[] inputValueDeserializer = new SerDe[Byte.MAX_VALUE]; - static { - ArrayList fieldNameArray = new ArrayList(); - for (Utilities.ReduceField r : Utilities.ReduceField.values()) { - fieldNameArray.add(r.toString()); - } - fieldNames = fieldNameArray.toArray(new String[0]); - } TableDesc keyTableDesc; TableDesc[] valueTableDesc; @@ -148,7 +141,7 @@ ois.add(valueObjectInspector[tag]); ois.add(PrimitiveObjectInspectorFactory.writableByteObjectInspector); rowObjectInspector[tag] = ObjectInspectorFactory - .getStandardStructObjectInspector(Arrays.asList(fieldNames), ois); + .getStandardStructObjectInspector(Utilities.fieldNameList, ois); } } catch (Exception e) { throw new RuntimeException(e); @@ -176,7 +169,7 @@ private BytesWritable groupKey; - ArrayList row = new ArrayList(3); + List row = new ArrayList(3); ByteWritable tag = new ByteWritable(); public void reduce(Object key, Iterator values, OutputCollector output, Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRUnion1.java @@ -226,6 +226,15 @@ // future Map, GenMapRedCtx> mapCurrCtx = ctx.getMapCurrCtx(); + if (union.getConf().isAllInputsInSameReducer()) { + // All inputs of this UnionOperator are in the same Reducer. + // We do not need to break the operator tree. + mapCurrCtx.put((Operator) nd, + new GenMapRedCtx(ctx.getCurrTask(), ctx.getCurrTopOp(), + ctx.getCurrAliasId())); + return null; + } + UnionParseContext uPrsCtx = uCtx.getUnionParseContext(union); ctx.setCurrUnionOp(union); Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.exec.ConditionalTask; +import org.apache.hadoop.hive.ql.exec.DemuxOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.OperatorFactory; @@ -113,7 +114,8 @@ || currTask.getParentTasks().isEmpty())) { rootTasks.add(currTask); } - if (reducer.getClass() == JoinOperator.class) { + if (reducer.getClass() == JoinOperator.class || + reducer.getClass() == DemuxOperator.class) { plan.setNeedsTagging(true); } @@ -158,7 +160,8 @@ plan.setNumReduceTasks(desc.getNumReducers()); - if (reducer.getClass() == JoinOperator.class) { + if (reducer.getClass() == JoinOperator.class || + reducer.getClass() == DemuxOperator.class) { plan.setNeedsTagging(true); } @@ -946,7 +949,8 @@ // TODO: Allocate work to remove the temporary files and make that // dependent on the redTask - if (reducer.getClass() == JoinOperator.class) { + if (reducer.getClass() == JoinOperator.class || + reducer.getClass() == DemuxOperator.class) { cplan.setNeedsTagging(true); } } Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -22,6 +22,8 @@ import java.util.List; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.optimizer.correlation.CorrelationOptimizer; +import org.apache.hadoop.hive.ql.optimizer.correlation.ReduceSinkDeDuplication; import org.apache.hadoop.hive.ql.optimizer.index.RewriteGBUsingIndex; import org.apache.hadoop.hive.ql.optimizer.lineage.Generator; import org.apache.hadoop.hive.ql.optimizer.listbucketingpruner.ListBucketingPruner; @@ -103,6 +105,11 @@ if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVELIMITOPTENABLE)) { transformations.add(new GlobalLimitOptimizer()); } + if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCORRELATION) && + !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEGROUPBYSKEW) && + !HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME)) { + transformations.add(new CorrelationOptimizer()); + } transformations.add(new SimpleFetchOptimizer()); // must be called last } Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkDeDuplication.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkDeDuplication.java +++ /dev/null @@ -1,790 +0,0 @@ -/** - * 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.lang.reflect.Array; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; - -import org.apache.hadoop.hive.ql.exec.ColumnInfo; -import org.apache.hadoop.hive.ql.exec.ExtractOperator; -import org.apache.hadoop.hive.ql.exec.FilterOperator; -import org.apache.hadoop.hive.ql.exec.ForwardOperator; -import org.apache.hadoop.hive.ql.exec.GroupByOperator; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -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.RowSchema; -import org.apache.hadoop.hive.ql.exec.ScriptOperator; -import org.apache.hadoop.hive.ql.exec.SelectOperator; -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.parse.OpParseContext; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.RowResolver; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.AggregationDesc; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; -import org.apache.hadoop.hive.ql.plan.GroupByDesc; -import org.apache.hadoop.hive.ql.plan.JoinCondDesc; -import org.apache.hadoop.hive.ql.plan.JoinDesc; -import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; -import org.apache.hadoop.hive.ql.plan.SelectDesc; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; - -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVECONVERTJOIN; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASK; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATIONMINREDUCER; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESCRIPTOPERATORTRUST; - -/** - * If two reducer sink operators share the same partition/sort columns and order, - * they can be merged. This should happen after map join optimization because map - * join optimization will remove reduce sink operators. - * - * This optimizer removes/replaces child-RS (not parent) which is safer way for DefaultGraphWalker. - */ -public class ReduceSinkDeDuplication implements Transform{ - - private static final String RS = ReduceSinkOperator.getOperatorName(); - private static final String GBY = GroupByOperator.getOperatorName(); - private static final String JOIN = JoinOperator.getOperatorName(); - - protected ParseContext pGraphContext; - - @Override - public ParseContext transform(ParseContext pctx) throws SemanticException { - pGraphContext = pctx; - - // generate pruned column list for all relevant operators - ReduceSinkDeduplicateProcCtx cppCtx = new ReduceSinkDeduplicateProcCtx(pGraphContext); - - // for auto convert map-joins, it not safe to dedup in here (todo) - boolean mergeJoins = !pctx.getConf().getBoolVar(HIVECONVERTJOIN) && - !pctx.getConf().getBoolVar(HIVECONVERTJOINNOCONDITIONALTASK); - - // If multiple rules can be matched with same cost, last rule will be choosen as a processor - // see DefaultRuleDispatcher#dispatch() - Map opRules = new LinkedHashMap(); - opRules.put(new RuleRegExp("R1", RS + "%.*%" + RS + "%"), - ReduceSinkDeduplicateProcFactory.getReducerReducerProc()); - opRules.put(new RuleRegExp("R2", RS + "%" + GBY + "%.*%" + RS + "%"), - ReduceSinkDeduplicateProcFactory.getGroupbyReducerProc()); - if (mergeJoins) { - opRules.put(new RuleRegExp("R3", JOIN + "%.*%" + RS + "%"), - ReduceSinkDeduplicateProcFactory.getJoinReducerProc()); - } - // TODO RS+JOIN - - // The dispatcher fires the processor corresponding to the closest matching - // rule and passes the context along - Dispatcher disp = new DefaultRuleDispatcher(ReduceSinkDeduplicateProcFactory - .getDefaultProc(), opRules, cppCtx); - GraphWalker ogw = new DefaultGraphWalker(disp); - - // Create a list of topop nodes - ArrayList topNodes = new ArrayList(); - topNodes.addAll(pGraphContext.getTopOps().values()); - ogw.startWalking(topNodes, null); - return pGraphContext; - } - - class ReduceSinkDeduplicateProcCtx implements NodeProcessorCtx { - - ParseContext pctx; - - // For queries using script, the optimization cannot be applied without user's confirmation - // If script preserves alias and value for columns related to keys, user can set this true - boolean trustScript; - - // This is min number of reducer for deduped RS to avoid query executed on - // too small number of reducers. For example, queries GroupBy+OrderBy can be executed by - // only one reducer if this configuration does not prevents - int minReducer; - Set> removedOps; - - public ReduceSinkDeduplicateProcCtx(ParseContext pctx) { - removedOps = new HashSet>(); - trustScript = pctx.getConf().getBoolVar(HIVESCRIPTOPERATORTRUST); - minReducer = pctx.getConf().getIntVar(HIVEOPTREDUCEDEDUPLICATIONMINREDUCER); - this.pctx = pctx; - } - - public boolean contains(Operator rsOp) { - return removedOps.contains(rsOp); - } - - public boolean addRemovedOperator(Operator rsOp) { - return removedOps.add(rsOp); - } - - public ParseContext getPctx() { - return pctx; - } - - public void setPctx(ParseContext pctx) { - this.pctx = pctx; - } - } - - static class ReduceSinkDeduplicateProcFactory { - - public static NodeProcessor getReducerReducerProc() { - return new ReducerReducerProc(); - } - - public static NodeProcessor getGroupbyReducerProc() { - return new GroupbyReducerProc(); - } - - public static NodeProcessor getJoinReducerProc() { - return new JoinReducerProc(); - } - - public static NodeProcessor getDefaultProc() { - return new DefaultProc(); - } - } - - /* - * do nothing. - */ - static class DefaultProc implements NodeProcessor { - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, - Object... nodeOutputs) throws SemanticException { - return null; - } - } - - public abstract static class AbstractReducerReducerProc implements NodeProcessor { - - ReduceSinkDeduplicateProcCtx dedupCtx; - - protected boolean trustScript() { - return dedupCtx.trustScript; - } - - protected int minReducer() { - return dedupCtx.minReducer; - } - - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, - Object... nodeOutputs) throws SemanticException { - dedupCtx = (ReduceSinkDeduplicateProcCtx) procCtx; - if (dedupCtx.contains((Operator) nd)) { - return false; - } - ReduceSinkOperator cRS = (ReduceSinkOperator) nd; - Operator child = getSingleChild(cRS); - if (child instanceof JoinOperator) { - return false; // not supported - } - ParseContext pctx = dedupCtx.getPctx(); - if (child instanceof GroupByOperator) { - GroupByOperator cGBY = (GroupByOperator) child; - if (!hasGroupingSet(cRS) && !cGBY.getConf().isGroupingSetsPresent()) { - return process(cRS, cGBY, pctx); - } - return false; - } - if (child instanceof ExtractOperator) { - return process(cRS, pctx); - } - return false; - } - - private boolean hasGroupingSet(ReduceSinkOperator cRS) { - GroupByOperator cGBYm = getSingleParent(cRS, GroupByOperator.class); - if (cGBYm != null && cGBYm.getConf().isGroupingSetsPresent()) { - return true; - } - return false; - } - - protected Operator getSingleParent(Operator operator) { - List> parents = operator.getParentOperators(); - if (parents != null && parents.size() == 1) { - return parents.get(0); - } - return null; - } - - protected Operator getSingleChild(Operator operator) { - List> children = operator.getChildOperators(); - if (children != null && children.size() == 1) { - return children.get(0); - } - return null; - } - - protected T getSingleParent(Operator operator, Class type) { - Operator parent = getSingleParent(operator); - return type.isInstance(parent) ? (T)parent : null; - } - - protected abstract Object process(ReduceSinkOperator cRS, ParseContext context) - throws SemanticException; - - protected abstract Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, - ParseContext context) throws SemanticException; - - protected Operator getStartForGroupBy(ReduceSinkOperator cRS) { - Operator parent = getSingleParent(cRS); - return parent instanceof GroupByOperator ? parent : cRS; // skip map-aggr GBY - } - - // for JOIN-RS case, it's not possible generally to merge if child has - // more key/partition columns than parents - protected boolean merge(ReduceSinkOperator cRS, JoinOperator pJoin, int minReducer) - throws SemanticException { - List> parents = pJoin.getParentOperators(); - ReduceSinkOperator[] pRSs = parents.toArray(new ReduceSinkOperator[parents.size()]); - ReduceSinkDesc cRSc = cRS.getConf(); - ReduceSinkDesc pRS0c = pRSs[0].getConf(); - if (cRSc.getKeyCols().size() > pRS0c.getKeyCols().size()) { - return false; - } - if (cRSc.getPartitionCols().size() > pRS0c.getPartitionCols().size()) { - return false; - } - Integer moveReducerNumTo = checkNumReducer(cRSc.getNumReducers(), pRS0c.getNumReducers()); - if (moveReducerNumTo == null || - moveReducerNumTo > 0 && cRSc.getNumReducers() < minReducer) { - return false; - } - - Integer moveRSOrderTo = checkOrder(cRSc.getOrder(), pRS0c.getOrder()); - if (moveRSOrderTo == null) { - return false; - } - - boolean[] sorted = getSortedTags(pJoin); - - int cKeySize = cRSc.getKeyCols().size(); - for (int i = 0; i < cKeySize; i++) { - ExprNodeDesc cexpr = cRSc.getKeyCols().get(i); - ExprNodeDesc[] pexprs = new ExprNodeDesc[pRSs.length]; - for (int tag = 0; tag < pRSs.length; tag++) { - pexprs[tag] = pRSs[tag].getConf().getKeyCols().get(i); - } - int found = indexOf(cexpr, pexprs, cRS, pRSs, sorted); - if (found < 0) { - return false; - } - } - int cPartSize = cRSc.getPartitionCols().size(); - for (int i = 0; i < cPartSize; i++) { - ExprNodeDesc cexpr = cRSc.getPartitionCols().get(i); - ExprNodeDesc[] pexprs = new ExprNodeDesc[pRSs.length]; - for (int tag = 0; tag < pRSs.length; tag++) { - pexprs[tag] = pRSs[tag].getConf().getPartitionCols().get(i); - } - int found = indexOf(cexpr, pexprs, cRS, pRSs, sorted); - if (found < 0) { - return false; - } - } - - if (moveReducerNumTo > 0) { - for (ReduceSinkOperator pRS : pRSs) { - pRS.getConf().setNumReducers(cRS.getConf().getNumReducers()); - } - } - return true; - } - - private boolean[] getSortedTags(JoinOperator joinOp) { - boolean[] result = new boolean[joinOp.getParentOperators().size()]; - for (int tag = 0; tag < result.length; tag++) { - result[tag] = isSortedTag(joinOp, tag); - } - return result; - } - - // for left outer joins, left alias is sorted but right alias might be not - // (nulls, etc.). vice versa. - private boolean isSortedTag(JoinOperator joinOp, int tag) { - for (JoinCondDesc cond : joinOp.getConf().getConds()) { - switch (cond.getType()) { - case JoinDesc.LEFT_OUTER_JOIN: - if (cond.getRight() == tag) { - return false; - } - continue; - case JoinDesc.RIGHT_OUTER_JOIN: - if (cond.getLeft() == tag) { - return false; - } - continue; - case JoinDesc.FULL_OUTER_JOIN: - if (cond.getLeft() == tag || cond.getRight() == tag) { - return false; - } - } - } - return true; - } - - private int indexOf(ExprNodeDesc cexpr, ExprNodeDesc[] pexprs, Operator child, - Operator[] parents, boolean[] sorted) throws SemanticException { - for (int tag = 0; tag < parents.length; tag++) { - if (sorted[tag] && - pexprs[tag].isSame(ExprNodeDescUtils.backtrack(cexpr, child, parents[tag]))) { - return tag; - } - } - return -1; - } - - /** - * Current RSDedup remove/replace child RS. So always copies - * more specific part of configurations of child RS to that of parent RS. - */ - protected boolean merge(ReduceSinkOperator cRS, ReduceSinkOperator pRS, int minReducer) - throws SemanticException { - int[] result = checkStatus(cRS, pRS, minReducer); - if (result == null) { - return false; - } - if (result[0] > 0) { - ArrayList childKCs = cRS.getConf().getKeyCols(); - pRS.getConf().setKeyCols(ExprNodeDescUtils.backtrack(childKCs, cRS, pRS)); - } - if (result[1] > 0) { - ArrayList childPCs = cRS.getConf().getPartitionCols(); - pRS.getConf().setPartitionCols(ExprNodeDescUtils.backtrack(childPCs, cRS, pRS)); - } - if (result[2] > 0) { - pRS.getConf().setOrder(cRS.getConf().getOrder()); - } - if (result[3] > 0) { - pRS.getConf().setNumReducers(cRS.getConf().getNumReducers()); - } - return true; - } - - /** - * Returns merge directions between two RSs for criterias (ordering, number of reducers, - * reducer keys, partition keys). Returns null if any of categories is not mergeable. - * - * Values for each index can be -1, 0, 1 - * 1. 0 means two configuration in the category is the same - * 2. for -1, configuration of parent RS is more specific than child RS - * 3. for 1, configuration of child RS is more specific than parent RS - */ - private int[] checkStatus(ReduceSinkOperator cRS, ReduceSinkOperator pRS, int minReducer) - throws SemanticException { - ReduceSinkDesc cConf = cRS.getConf(); - ReduceSinkDesc pConf = pRS.getConf(); - Integer moveRSOrderTo = checkOrder(cConf.getOrder(), pConf.getOrder()); - if (moveRSOrderTo == null) { - return null; - } - Integer moveReducerNumTo = checkNumReducer(cConf.getNumReducers(), pConf.getNumReducers()); - if (moveReducerNumTo == null || - moveReducerNumTo > 0 && cConf.getNumReducers() < minReducer) { - return null; - } - List ckeys = cConf.getKeyCols(); - List pkeys = pConf.getKeyCols(); - Integer moveKeyColTo = checkExprs(ckeys, pkeys, cRS, pRS); - if (moveKeyColTo == null) { - return null; - } - List cpars = cConf.getPartitionCols(); - List ppars = pConf.getPartitionCols(); - Integer movePartitionColTo = checkExprs(cpars, ppars, cRS, pRS); - if (movePartitionColTo == null) { - return null; - } - return new int[] {moveKeyColTo, movePartitionColTo, moveRSOrderTo, moveReducerNumTo}; - } - - /** - * Overlapping part of keys should be the same between parent and child. - * And if child has more keys than parent, non-overlapping part of keys - * should be backtrackable to parent. - */ - private Integer checkExprs(List ckeys, List pkeys, - ReduceSinkOperator cRS, ReduceSinkOperator pRS) throws SemanticException { - Integer moveKeyColTo = 0; - if (ckeys == null || ckeys.isEmpty()) { - if (pkeys != null && !pkeys.isEmpty()) { - moveKeyColTo = -1; - } - } else { - if (pkeys == null || pkeys.isEmpty()) { - for (ExprNodeDesc ckey : ckeys) { - if (ExprNodeDescUtils.backtrack(ckey, cRS, pRS) == null) { - // cKey is not present in parent - return null; - } - } - moveKeyColTo = 1; - } else { - moveKeyColTo = sameKeys(ckeys, pkeys, cRS, pRS); - } - } - return moveKeyColTo; - } - - // backtrack key exprs of child to parent and compare it with parent's - protected Integer sameKeys(List cexprs, List pexprs, - Operator child, Operator parent) throws SemanticException { - int common = Math.min(cexprs.size(), pexprs.size()); - int limit = Math.max(cexprs.size(), pexprs.size()); - int i = 0; - for (; i < common; i++) { - ExprNodeDesc pexpr = pexprs.get(i); - ExprNodeDesc cexpr = ExprNodeDescUtils.backtrack(cexprs.get(i), child, parent); - if (cexpr == null || !pexpr.isSame(cexpr)) { - return null; - } - } - for (;i < limit; i++) { - if (cexprs.size() > pexprs.size()) { - if (ExprNodeDescUtils.backtrack(cexprs.get(i), child, parent) == null) { - // cKey is not present in parent - return null; - } - } - } - return Integer.valueOf(cexprs.size()).compareTo(pexprs.size()); - } - - // order of overlapping keys should be exactly the same - protected Integer checkOrder(String corder, String porder) { - if (corder == null || corder.trim().equals("")) { - if (porder == null || porder.trim().equals("")) { - return 0; - } - return -1; - } - if (porder == null || porder.trim().equals("")) { - return 1; - } - corder = corder.trim(); - porder = porder.trim(); - int target = Math.min(corder.length(), porder.length()); - if (!corder.substring(0, target).equals(porder.substring(0, target))) { - return null; - } - return Integer.valueOf(corder.length()).compareTo(porder.length()); - } - - /** - * If number of reducers for RS is -1, the RS can have any number of reducers. - * It's generally true except for order-by or forced bucketing cases. - * if both of num-reducers are not -1, those number should be the same. - */ - protected Integer checkNumReducer(int creduce, int preduce) { - if (creduce < 0) { - if (preduce < 0) { - return 0; - } - return -1; - } - if (preduce < 0) { - return 1; - } - if (creduce != preduce) { - return null; - } - return 0; - } - - protected > T findPossibleParent(Operator start, Class target, - boolean trustScript) throws SemanticException { - T[] parents = findPossibleParents(start, target, trustScript); - return parents != null && parents.length == 1 ? parents[0] : null; - } - - @SuppressWarnings("unchecked") - protected > T[] findPossibleParents(Operator start, Class target, - boolean trustScript) { - Operator cursor = getSingleParent(start); - for (; cursor != null; cursor = getSingleParent(cursor)) { - if (target.isAssignableFrom(cursor.getClass())) { - T[] array = (T[]) Array.newInstance(target, 1); - array[0] = (T) cursor; - return array; - } - if (cursor instanceof JoinOperator) { - return findParents((JoinOperator) cursor, target); - } - if (cursor instanceof ScriptOperator && !trustScript) { - return null; - } - if (!(cursor instanceof SelectOperator - || cursor instanceof FilterOperator - || cursor instanceof ExtractOperator - || cursor instanceof ForwardOperator - || cursor instanceof ScriptOperator - || cursor instanceof ReduceSinkOperator)) { - return null; - } - } - return null; - } - - @SuppressWarnings("unchecked") - private > T[] findParents(JoinOperator join, Class target) { - List> parents = join.getParentOperators(); - T[] result = (T[]) Array.newInstance(target, parents.size()); - for (int tag = 0; tag < result.length; tag++) { - Operator cursor = parents.get(tag); - for (; cursor != null; cursor = getSingleParent(cursor)) { - if (target.isAssignableFrom(cursor.getClass())) { - result[tag] = (T) cursor; - break; - } - } - if (result[tag] == null) { - throw new IllegalStateException("failed to find " + target.getSimpleName() - + " from " + join + " on tag " + tag); - } - } - return result; - } - - protected SelectOperator replaceReduceSinkWithSelectOperator(ReduceSinkOperator childRS, - ParseContext context) throws SemanticException { - SelectOperator select = replaceOperatorWithSelect(childRS, context); - select.getConf().setOutputColumnNames(childRS.getConf().getOutputValueColumnNames()); - select.getConf().setColList(childRS.getConf().getValueCols()); - return select; - } - - // replace the cRS to SEL operator - // If child if cRS is EXT, EXT also should be removed - private SelectOperator replaceOperatorWithSelect(Operator operator, ParseContext context) - throws SemanticException { - RowResolver inputRR = context.getOpParseCtx().get(operator).getRowResolver(); - SelectDesc select = new SelectDesc(null, null); - - Operator parent = getSingleParent(operator); - Operator child = getSingleChild(operator); - - parent.getChildOperators().clear(); - - SelectOperator sel = (SelectOperator) putOpInsertMap( - OperatorFactory.getAndMakeChild(select, new RowSchema(inputRR - .getColumnInfos()), parent), inputRR, context); - - sel.setColumnExprMap(operator.getColumnExprMap()); - - sel.setChildOperators(operator.getChildOperators()); - for (Operator ch : operator.getChildOperators()) { - ch.replaceParent(operator, sel); - } - if (child instanceof ExtractOperator) { - removeOperator(child, getSingleChild(child), sel, context); - dedupCtx.addRemovedOperator(child); - } - operator.setChildOperators(null); - operator.setParentOperators(null); - dedupCtx.addRemovedOperator(operator); - return sel; - } - - protected void removeReduceSinkForGroupBy(ReduceSinkOperator cRS, GroupByOperator cGBYr, - ParseContext context) throws SemanticException { - - Operator parent = getSingleParent(cRS); - - if (parent instanceof GroupByOperator) { - // pRS-cGBYm-cRS-cGBYr (map aggregation) --> pRS-cGBYr(COMPLETE) - // copies desc of cGBYm to cGBYr and remove cGBYm and cRS - GroupByOperator cGBYm = (GroupByOperator) parent; - - cGBYr.getConf().setKeys(cGBYm.getConf().getKeys()); - cGBYr.getConf().setAggregators(cGBYm.getConf().getAggregators()); - for (AggregationDesc aggr : cGBYm.getConf().getAggregators()) { - aggr.setMode(GenericUDAFEvaluator.Mode.COMPLETE); - } - cGBYr.setColumnExprMap(cGBYm.getColumnExprMap()); - cGBYr.setSchema(cGBYm.getSchema()); - RowResolver resolver = context.getOpParseCtx().get(cGBYm).getRowResolver(); - context.getOpParseCtx().get(cGBYr).setRowResolver(resolver); - } else { - // pRS-cRS-cGBYr (no map aggregation) --> pRS-cGBYr(COMPLETE) - // revert expressions of cGBYr to that of cRS - cGBYr.getConf().setKeys(ExprNodeDescUtils.backtrack(cGBYr.getConf().getKeys(), cGBYr, cRS)); - for (AggregationDesc aggr : cGBYr.getConf().getAggregators()) { - aggr.setParameters(ExprNodeDescUtils.backtrack(aggr.getParameters(), cGBYr, cRS)); - } - - Map oldMap = cGBYr.getColumnExprMap(); - RowResolver oldRR = context.getOpParseCtx().get(cGBYr).getRowResolver(); - - Map newMap = new HashMap(); - RowResolver newRR = new RowResolver(); - - List outputCols = cGBYr.getConf().getOutputColumnNames(); - for (int i = 0; i < outputCols.size(); i++) { - String colName = outputCols.get(i); - String[] nm = oldRR.reverseLookup(colName); - ColumnInfo colInfo = oldRR.get(nm[0], nm[1]); - newRR.put(nm[0], nm[1], colInfo); - ExprNodeDesc colExpr = ExprNodeDescUtils.backtrack(oldMap.get(colName), cGBYr, cRS); - if (colExpr != null) { - newMap.put(colInfo.getInternalName(), colExpr); - } - } - cGBYr.setColumnExprMap(newMap); - cGBYr.setSchema(new RowSchema(newRR.getColumnInfos())); - context.getOpParseCtx().get(cGBYr).setRowResolver(newRR); - } - cGBYr.getConf().setMode(GroupByDesc.Mode.COMPLETE); - - removeOperator(cRS, cGBYr, parent, context); - dedupCtx.addRemovedOperator(cRS); - - if (parent instanceof GroupByOperator) { - removeOperator(parent, cGBYr, getSingleParent(parent), context); - dedupCtx.addRemovedOperator(cGBYr); - } - } - - private void removeOperator(Operator target, Operator child, Operator parent, - ParseContext context) { - for (Operator aparent : target.getParentOperators()) { - aparent.replaceChild(target, child); - } - for (Operator achild : target.getChildOperators()) { - achild.replaceParent(target, parent); - } - target.setChildOperators(null); - target.setParentOperators(null); - context.getOpParseCtx().remove(target); - } - - private Operator putOpInsertMap(Operator op, RowResolver rr, - ParseContext context) { - OpParseContext ctx = new OpParseContext(rr); - context.getOpParseCtx().put(op, ctx); - return op; - } - } - - static class GroupbyReducerProc extends AbstractReducerReducerProc { - - // pRS-pGBY-cRS - public Object process(ReduceSinkOperator cRS, ParseContext context) - throws SemanticException { - GroupByOperator pGBY = findPossibleParent(cRS, GroupByOperator.class, trustScript()); - if (pGBY == null) { - return false; - } - ReduceSinkOperator pRS = findPossibleParent(pGBY, ReduceSinkOperator.class, trustScript()); - if (pRS != null && merge(cRS, pRS, minReducer())) { - replaceReduceSinkWithSelectOperator(cRS, context); - return true; - } - return false; - } - - // pRS-pGBY-cRS-cGBY - public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, ParseContext context) - throws SemanticException { - Operator start = getStartForGroupBy(cRS); - GroupByOperator pGBY = findPossibleParent(start, GroupByOperator.class, trustScript()); - if (pGBY == null) { - return false; - } - ReduceSinkOperator pRS = getSingleParent(pGBY, ReduceSinkOperator.class); - if (pRS != null && merge(cRS, pRS, minReducer())) { - removeReduceSinkForGroupBy(cRS, cGBY, context); - return true; - } - return false; - } - } - - static class JoinReducerProc extends AbstractReducerReducerProc { - - // pRS-pJOIN-cRS - public Object process(ReduceSinkOperator cRS, ParseContext context) - throws SemanticException { - JoinOperator pJoin = findPossibleParent(cRS, JoinOperator.class, trustScript()); - if (pJoin != null && merge(cRS, pJoin, minReducer())) { - pJoin.getConf().setFixedAsSorted(true); - replaceReduceSinkWithSelectOperator(cRS, context); - return true; - } - return false; - } - - // pRS-pJOIN-cRS-cGBY - public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, ParseContext context) - throws SemanticException { - Operator start = getStartForGroupBy(cRS); - JoinOperator pJoin = findPossibleParent(start, JoinOperator.class, trustScript()); - if (pJoin != null && merge(cRS, pJoin, minReducer())) { - pJoin.getConf().setFixedAsSorted(true); - removeReduceSinkForGroupBy(cRS, cGBY, context); - return true; - } - return false; - } - } - - static class ReducerReducerProc extends AbstractReducerReducerProc { - - // pRS-cRS - public Object process(ReduceSinkOperator cRS, ParseContext context) - throws SemanticException { - ReduceSinkOperator pRS = findPossibleParent(cRS, ReduceSinkOperator.class, trustScript()); - if (pRS != null && merge(cRS, pRS, minReducer())) { - replaceReduceSinkWithSelectOperator(cRS, context); - return true; - } - return false; - } - - // pRS-cRS-cGBY - public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, ParseContext context) - throws SemanticException { - Operator start = getStartForGroupBy(cRS); - ReduceSinkOperator pRS = findPossibleParent(start, ReduceSinkOperator.class, trustScript()); - if (pRS != null && merge(cRS, pRS, minReducer())) { - removeReduceSinkForGroupBy(cRS, cGBY, context); - return true; - } - return false; - } - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/AbstractCorrelationProcCtx.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/AbstractCorrelationProcCtx.java @@ -0,0 +1,73 @@ +/** + * 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.correlation; + +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATIONMINREDUCER; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESCRIPTOPERATORTRUST; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.parse.ParseContext; + +abstract class AbstractCorrelationProcCtx implements NodeProcessorCtx { + private ParseContext pctx; + // For queries using script, the optimization cannot be applied without user's confirmation + // If script preserves alias and value for columns related to keys, user can set this true + private final boolean trustScript; + + // This is min number of reducer for deduped RS to avoid query executed on + // too small number of reducers. For example, queries GroupBy+OrderBy can be executed by + // only one reducer if this configuration does not prevents + private final int minReducer; + private final Set> removedOps; + + public AbstractCorrelationProcCtx(ParseContext pctx) { + removedOps = new HashSet>(); + trustScript = pctx.getConf().getBoolVar(HIVESCRIPTOPERATORTRUST); + minReducer = pctx.getConf().getIntVar(HIVEOPTREDUCEDEDUPLICATIONMINREDUCER); + this.pctx = pctx; + } + + public ParseContext getPctx() { + return pctx; + } + + public void setPctx(ParseContext pctx) { + this.pctx = pctx; + } + + public boolean trustScript() { + return trustScript; + } + + public int minReducer() { + return minReducer; + } + + public boolean hasBeenRemoved(Operator rsOp) { + return removedOps.contains(rsOp); + } + + public boolean addRemovedOperator(Operator rsOp) { + return removedOps.add(rsOp); + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java @@ -0,0 +1,714 @@ +/** + * 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.correlation; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.GroupByOperator; +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.PTFOperator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.UnionOperator; +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.Table; +import org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor; +import org.apache.hadoop.hive.ql.optimizer.Transform; +import org.apache.hadoop.hive.ql.optimizer.physical.CommonJoinTaskDispatcher; +import org.apache.hadoop.hive.ql.parse.OpParseContext; +import org.apache.hadoop.hive.ql.parse.ParseContext; +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.ExprNodeDescUtils; +import org.apache.hadoop.hive.ql.plan.JoinCondDesc; +import org.apache.hadoop.hive.ql.plan.JoinDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; + +/** + * Implementation of Correlation Optimizer. This optimizer is based on + * the paper "YSmart: Yet Another SQL-to-MapReduce Translator" + * (Rubao Lee, Tian Luo, Yin Huai, Fusheng Wang, Yongqiang He, and Xiaodong Zhang) + * (http://www.cse.ohio-state.edu/hpcs/WWW/HTML/publications/papers/TR-11-7.pdf). + * Correlation Optimizer detects if ReduceSinkOperators share same keys. + * Then, it will transform the query plan tree (operator tree) by exploiting + * detected correlations. For details, see the original paper of YSmart. + * + * Test queries associated with this optimizer are correlationoptimizer1.q to + * correlationoptimizer14.q + */ +public class CorrelationOptimizer implements Transform { + + private static final Log LOG = LogFactory.getLog(CorrelationOptimizer.class.getName()); + + private boolean abort; // if correlation optimizer will not try to optimize this query + + private ParseContext pCtx; + + //Join operators which may be converted by CommonJoinResolver; + private final Set> skipedJoinOperators; + + public CorrelationOptimizer() { + super(); + pCtx = null; + skipedJoinOperators = new HashSet>(); + abort = false; + } + + private void findPossibleAutoConvertedJoinOperators() throws SemanticException { + // Guess if CommonJoinResolver will work. If CommonJoinResolver may + // convert a join operation, correlation optimizer will not merge that join. + // TODO: If hive.auto.convert.join.noconditionaltask=true, for a JoinOperator + // that has both intermediate tables and query input tables as input tables, + // we should be able to guess if this JoinOperator will be converted to a MapJoin + // based on hive.auto.convert.join.noconditionaltask.size. + for (JoinOperator joinOp: pCtx.getJoinContext().keySet()) { + boolean isAbleToGuess = true; + boolean mayConvert = false; + // Get total size and individual alias's size + long aliasTotalKnownInputSize = 0; + Map aliasToSize = new HashMap(); + Map posToAlias = new HashMap(); + for (Operator op: joinOp.getParentOperators()) { + TableScanOperator tsop = CorrelationUtilities.findTableScanOperator(op); + if (tsop == null) { + isAbleToGuess = false; + break; + } + + Table table = pCtx.getTopToTable().get(tsop); + String alias = tsop.getConf().getAlias(); + posToAlias.put(joinOp.getParentOperators().indexOf(op), alias); + if (table == null) { + // table should not be null. + throw new SemanticException("The table of " + + tsop.getName() + " " + tsop.getIdentifier() + + " is null, which is not expected."); + } + + Path p = table.getPath(); + FileSystem fs = null; + ContentSummary resultCs = null; + try { + fs = table.getPath().getFileSystem(pCtx.getConf()); + resultCs = fs.getContentSummary(p); + } catch (IOException e) { + LOG.warn("Encounter a error while querying content summary of table " + + table.getCompleteName() + " from FileSystem. " + + "Cannot guess if CommonJoinOperator will optimize " + + joinOp.getName() + " " + joinOp.getIdentifier()); + } + if (resultCs == null) { + isAbleToGuess = false; + break; + } + + long size = resultCs.getLength(); + aliasTotalKnownInputSize += size; + Long es = aliasToSize.get(alias); + if(es == null) { + es = new Long(0); + } + es += size; + aliasToSize.put(alias, es); + } + + if (!isAbleToGuess) { + LOG.info("Cannot guess if CommonJoinOperator will optimize " + + joinOp.getName() + " " + joinOp.getIdentifier()); + continue; + } + + JoinDesc joinDesc = joinOp.getConf(); + Byte[] order = joinDesc.getTagOrder(); + int numAliases = order.length; + HashSet bigTableCandidates = + MapJoinProcessor.getBigTableCandidates(joinDesc.getConds()); + if (bigTableCandidates == null) { + continue; + } + + String bigTableAlias = null; + long ThresholdOfSmallTblSizeSum = HiveConf.getLongVar(pCtx.getConf(), + HiveConf.ConfVars.HIVESMALLTABLESFILESIZE); + for (int i = 0; i < numAliases; i++) { + // this table cannot be big table + if (!bigTableCandidates.contains(i)) { + continue; + } + bigTableAlias = posToAlias.get(i); + if (!CommonJoinTaskDispatcher.cannotConvert(bigTableAlias, aliasToSize, + aliasTotalKnownInputSize, ThresholdOfSmallTblSizeSum)) { + mayConvert = true; + } + } + + if (mayConvert) { + LOG.info(joinOp.getName() + " " + joinOp.getIdentifier() + + " may be converted to MapJoin by CommonJoinResolver"); + skipedJoinOperators.add(joinOp); + } + } + } + + /** + * Detect correlations and transform the query tree. + * + * @param pactx + * current parse context + * @throws SemanticException + */ + public ParseContext transform(ParseContext pctx) throws SemanticException { + + pCtx = pctx; + + if (HiveConf.getBoolVar(pCtx.getConf(),HiveConf.ConfVars.HIVECONVERTJOIN)) { + findPossibleAutoConvertedJoinOperators(); + } + + // detect correlations + CorrelationNodeProcCtx corrCtx = new CorrelationNodeProcCtx(pCtx); + Map opRules = new LinkedHashMap(); + opRules.put(new RuleRegExp("R1", ReduceSinkOperator.getOperatorName() + "%"), + new CorrelationNodeProc()); + + Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, corrCtx); + GraphWalker ogw = new DefaultGraphWalker(disp); + + // Create a list of topOp nodes + List topNodes = new ArrayList(); + topNodes.addAll(pCtx.getTopOps().values()); + ogw.startWalking(topNodes, null); + // We have finished tree walking (correlation detection). + // We will first see if we need to abort (the operator tree has not been changed). + // If not, we will start to transform the operator tree. + abort = corrCtx.isAbort(); + if (abort) { + LOG.info("Abort. Reasons are ..."); + for (String reason : corrCtx.getAbortReasons()) { + LOG.info("-- " + reason); + } + } else { + // transform the operator tree + LOG.info("Begain query plan transformation based on intra-query correlations. " + + corrCtx.getCorrelations().size() + " correlation(s) to be applied"); + for (IntraQueryCorrelation correlation : corrCtx.getCorrelations()) { + QueryPlanTreeTransformation.applyCorrelation(pCtx, corrCtx, correlation); + } + } + return pCtx; + } + + private class CorrelationNodeProc implements NodeProcessor { + + private void analyzeReduceSinkOperatorsOfJoinOperator(JoinCondDesc[] joinConds, + List> rsOps, Operator curentRsOp, + Set correlatedRsOps) { + if (correlatedRsOps.contains((ReduceSinkOperator) curentRsOp)) { + return; + } + correlatedRsOps.add((ReduceSinkOperator) curentRsOp); + + int pos = rsOps.indexOf(curentRsOp); + for (int i = 0; i < joinConds.length; i++) { + JoinCondDesc joinCond = joinConds[i]; + int type = joinCond.getType(); + if (pos == joinCond.getLeft()) { + if (type == JoinDesc.INNER_JOIN || + type == JoinDesc.LEFT_OUTER_JOIN || + type == JoinDesc.LEFT_SEMI_JOIN) { + Operator newCurrentRsOps = rsOps.get(joinCond.getRight()); + analyzeReduceSinkOperatorsOfJoinOperator(joinConds, rsOps, newCurrentRsOps, + correlatedRsOps); + } + } else if (pos == joinCond.getRight()) { + if (type == JoinDesc.INNER_JOIN || type == JoinDesc.RIGHT_OUTER_JOIN) { + Operator newCurrentRsOps = rsOps.get(joinCond.getLeft()); + analyzeReduceSinkOperatorsOfJoinOperator(joinConds, rsOps, newCurrentRsOps, + correlatedRsOps); + } + } + } + } + + private boolean sameKeys(List k1, List k2) { + if (k1.size() != k2.size()) { + return false; + } + for (int i = 0; i < k1.size(); i++) { + ExprNodeDesc expr1 = k1.get(i); + ExprNodeDesc expr2 = k2.get(i); + if (expr1 == null) { + if (expr2 == null) { + continue; + } else { + return false; + } + } else { + if (!expr1.isSame(expr2)) { + return false; + } + } + } + return true; + } + + private boolean sameOrder(String order1, String order2) { + if (order1 == null || order1.trim().equals("")) { + if (order2 == null || order2.trim().equals("")) { + return true; + } + return false; + } + if (order2 == null || order2.trim().equals("")) { + return false; + } + order1 = order1.trim(); + order2 = order2.trim(); + if (!order1.equals(order2)) { + return false; + } + return true; + } + /** + * This method is used to recursively traverse the tree to find + * ReduceSinkOperators which share the same key columns and partitioning + * columns. Those ReduceSinkOperators are called correlated ReduceSinkOperaotrs. + * + * @param child The child of the current operator + * @param childKeyCols The key columns from the child operator + * @param childPartitionCols The partitioning columns from the child operator + * @param childRSOrder The sorting order of key columns from the child operator + * @param current The current operator we are visiting + * @param correlation The object keeps tracking the correlation + * @return + * @throws SemanticException + */ + private LinkedHashSet findCorrelatedReduceSinkOperators( + Operator child, + List childKeyCols, List childPartitionCols, + String childRSOrder, + Operator current, + IntraQueryCorrelation correlation) throws SemanticException { + + LOG.info("now detecting operator " + current.getIdentifier() + " " + current.getName()); + + LinkedHashSet correlatedReduceSinkOperators = + new LinkedHashSet(); + if (skipedJoinOperators.contains(current)) { + LOG.info(current.getName() + " " + current.getIdentifier() + + " may be converted to MapJoin by " + + "CommonJoinResolver. Correlation optimizer will not detect correlations" + + "involved in this operator"); + return correlatedReduceSinkOperators; + } + if (current.getParentOperators() == null) { + return correlatedReduceSinkOperators; + } + if (current instanceof PTFOperator) { + // Currently, we do not support PTF operator. + LOG.info("Currently, correlation optimizer does not support PTF operator."); + return correlatedReduceSinkOperators; + } + if (current instanceof UnionOperator) { + // If we get a UnionOperator, right now, we only handle it when + // we can find correlated ReduceSinkOperators from all inputs. + LinkedHashSet corrRSs = new LinkedHashSet(); + for (Operator parent : current.getParentOperators()) { + LinkedHashSet tmp = + findCorrelatedReduceSinkOperators( + current, childKeyCols, childPartitionCols, childRSOrder, parent, correlation); + if (tmp != null && tmp.size() > 0) { + corrRSs.addAll(tmp); + } else { + return correlatedReduceSinkOperators; + } + } + correlatedReduceSinkOperators.addAll(corrRSs); + UnionOperator union = (UnionOperator)current; + union.getConf().setAllInputsInSameReducer(true); + } else if (current.getColumnExprMap() == null && !(current instanceof ReduceSinkOperator)) { + for (Operator parent : current.getParentOperators()) { + correlatedReduceSinkOperators.addAll( + findCorrelatedReduceSinkOperators( + current, childKeyCols, childPartitionCols, childRSOrder, parent, correlation)); + } + } else if (current.getColumnExprMap() != null && !(current instanceof ReduceSinkOperator)) { + List backtrackedKeyCols = + ExprNodeDescUtils.backtrack(childKeyCols, child, current); + List backtrackedPartitionCols = + ExprNodeDescUtils.backtrack(childPartitionCols, child, current); + Set tableNeedToCheck = new HashSet(); + for (ExprNodeDesc expr: childKeyCols) { + if (!(expr instanceof ExprNodeColumnDesc)) { + return correlatedReduceSinkOperators; + } else { + String colName = ((ExprNodeColumnDesc)expr).getColumn(); + OpParseContext opCtx = pCtx.getOpParseCtx().get(current); + for (ColumnInfo cinfo : opCtx.getRowResolver().getColumnInfos()) { + if (colName.equals(cinfo.getInternalName())) { + tableNeedToCheck.add(cinfo.getTabAlias()); + } + } + } + } + if (current instanceof JoinOperator) { + LinkedHashSet correlatedRsOps = + new LinkedHashSet(); + for (Operator parent : current.getParentOperators()) { + Set tableNames = + pCtx.getOpParseCtx().get(parent).getRowResolver().getTableNames(); + for (String tbl : tableNames) { + if (tableNeedToCheck.contains(tbl)) { + correlatedRsOps.addAll(findCorrelatedReduceSinkOperators( + current, backtrackedKeyCols, backtrackedPartitionCols, childRSOrder, + parent, correlation)); + } + } + } + // If current is JoinOperaotr, we will stop to traverse the tree + // when any of parent ReduceSinkOperaotr of this JoinOperator is + // not considered as a correlated ReduceSinkOperator. + if (correlatedRsOps.size() == current.getParentOperators().size()) { + correlatedReduceSinkOperators.addAll(correlatedRsOps); + } else { + correlatedReduceSinkOperators.clear(); + } + } else { + for (Operator parent : current.getParentOperators()) { + correlatedReduceSinkOperators.addAll(findCorrelatedReduceSinkOperators( + current, backtrackedKeyCols, backtrackedPartitionCols, childRSOrder, + parent, correlation)); + } + } + } else if (current.getColumnExprMap() != null && current instanceof ReduceSinkOperator) { + ReduceSinkOperator rsop = (ReduceSinkOperator) current; + List backtrackedKeyCols = + ExprNodeDescUtils.backtrack(childKeyCols, child, current); + List backtrackedPartitionCols = + ExprNodeDescUtils.backtrack(childPartitionCols, child, current); + List rsKeyCols = rsop.getConf().getKeyCols(); + List rsPartitionCols = rsop.getConf().getPartitionCols(); + + // Two ReduceSinkOperators are correlated means that + // they have same sorting columns (key columns), same partitioning columns, + // same sorting orders, and no conflict on the numbers of reducers. + // TODO: we should relax this condition + // TODO: we need to handle aggregation functions with distinct keyword. In this case, + // distinct columns will be added to the key columns. + boolean isCorrelated = sameKeys(rsKeyCols, backtrackedKeyCols) && + sameOrder(rsop.getConf().getOrder(), childRSOrder) && + sameKeys(backtrackedPartitionCols, rsPartitionCols) && + correlation.adjustNumReducers(rsop.getConf().getNumReducers()); + GroupByOperator cGBY = + CorrelationUtilities.getSingleChild(rsop, GroupByOperator.class); + if (cGBY != null) { + if (CorrelationUtilities.hasGroupingSet(rsop) || + cGBY.getConf().isGroupingSetsPresent()) { + // Do not support grouping set right now + isCorrelated = false; + } + } + + if (isCorrelated) { + LOG.info("Operator " + current.getIdentifier() + " " + + current.getName() + " is correlated"); + Operator childOperator = + CorrelationUtilities.getSingleChild(current, true); + if (childOperator instanceof JoinOperator) { + JoinOperator joinOp = (JoinOperator) childOperator; + JoinCondDesc[] joinConds = joinOp.getConf().getConds(); + List> rsOps = joinOp.getParentOperators(); + LinkedHashSet correlatedRsOps = + new LinkedHashSet(); + analyzeReduceSinkOperatorsOfJoinOperator(joinConds, rsOps, current, correlatedRsOps); + correlatedReduceSinkOperators.addAll(correlatedRsOps); + } else { + correlatedReduceSinkOperators.add(rsop); + } + } else { + LOG.info("Operator " + current.getIdentifier() + " " + + current.getName() + " is not correlated"); + correlatedReduceSinkOperators.clear(); + } + } else { + LOG.error("ReduceSinkOperator " + current.getIdentifier() + " does not have ColumnExprMap"); + throw new SemanticException("CorrelationOptimizer cannot optimize this plan. " + + "ReduceSinkOperator " + current.getIdentifier() + + " does not have ColumnExprMap"); + } + return correlatedReduceSinkOperators; + } + + /** Start to exploit Job Flow Correlation from op. + * Example: here is the operator tree we have ... + * JOIN2 + * / \ + * RS4 RS5 + * / \ + * GBY1 JOIN1 + * | / \ + * RS1 RS2 RS3 + * The op will be RS4. If we can execute GBY1, JOIN1, and JOIN2 in + * the same reducer. This method will return [RS1, RS2, RS3]. + * @param op + * @param correlationCtx + * @param correlation + * @return + * @throws SemanticException + */ + private LinkedHashSet exploitJobFlowCorrelation(ReduceSinkOperator op, + CorrelationNodeProcCtx correlationCtx, IntraQueryCorrelation correlation) + throws SemanticException { + correlationCtx.addWalked(op); + correlation.addToAllReduceSinkOperators(op); + boolean shouldDetect = true; + LinkedHashSet reduceSinkOperators = + new LinkedHashSet(); + List keyCols = op.getConf().getKeyCols(); + List partitionCols = op.getConf().getPartitionCols(); + for (ExprNodeDesc key : keyCols) { + if (!(key instanceof ExprNodeColumnDesc)) { + shouldDetect = false; + } + } + for (ExprNodeDesc key : partitionCols) { + if (!(key instanceof ExprNodeColumnDesc)) { + shouldDetect = false; + } + } + GroupByOperator cGBY = + CorrelationUtilities.getSingleChild(op, GroupByOperator.class); + if (cGBY != null) { + if (CorrelationUtilities.hasGroupingSet(op) || + cGBY.getConf().isGroupingSetsPresent()) { + // Do not support grouping set right now + shouldDetect = false; + } + } + + if (shouldDetect) { + LinkedHashSet newReduceSinkOperators = + new LinkedHashSet(); + String sortOrder = op.getConf().getOrder(); + for (Operator parent : op.getParentOperators()) { + LOG.info("Operator " + op.getIdentifier() + + ": start detecting correlation from this operator"); + LinkedHashSet correlatedReduceSinkOperators = + findCorrelatedReduceSinkOperators(op, keyCols, partitionCols, + sortOrder, parent, correlation); + if (correlatedReduceSinkOperators.size() == 0) { + newReduceSinkOperators.add(op); + } else { + for (ReduceSinkOperator rsop : correlatedReduceSinkOperators) { + LinkedHashSet exploited = + exploitJobFlowCorrelation(rsop, correlationCtx, correlation); + if (exploited.size() == 0) { + newReduceSinkOperators.add(rsop); + } else { + newReduceSinkOperators.addAll(exploited); + } + } + } + } + reduceSinkOperators.addAll(newReduceSinkOperators); + } + return reduceSinkOperators; + } + + public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, + Object... nodeOutputs) throws SemanticException { + CorrelationNodeProcCtx corrCtx = (CorrelationNodeProcCtx) ctx; + ReduceSinkOperator op = (ReduceSinkOperator) nd; + + // Check if we have visited this operator + if (corrCtx.isWalked(op)) { + return null; + } + + LOG.info("Walk to operator " + op.getIdentifier() + " " + op.getName()); + + Operator child = CorrelationUtilities.getSingleChild(op, true); + if (!(child instanceof JoinOperator) && !(child instanceof GroupByOperator)) { + corrCtx.addWalked(op); + return null; + } + + // detect correlations + IntraQueryCorrelation correlation = new IntraQueryCorrelation(corrCtx.minReducer()); + List topReduceSinkOperators = + CorrelationUtilities.findSiblingReduceSinkOperators(op); + List bottomReduceSinkOperators = new ArrayList(); + // Adjust the number of reducers of this correlation based on + // those top layer ReduceSinkOperators. + for (ReduceSinkOperator rsop : topReduceSinkOperators) { + if (!correlation.adjustNumReducers(rsop.getConf().getNumReducers())) { + // If we have a conflict on the number of reducers, we will not optimize + // this plan from here. + corrCtx.addWalked(op); + return null; + } + } + for (ReduceSinkOperator rsop : topReduceSinkOperators) { + LinkedHashSet thisBottomReduceSinkOperators = + exploitJobFlowCorrelation(rsop, corrCtx, correlation); + if (thisBottomReduceSinkOperators.size() == 0) { + thisBottomReduceSinkOperators.add(rsop); + } + bottomReduceSinkOperators.addAll(thisBottomReduceSinkOperators); + } + + if (!topReduceSinkOperators.containsAll(bottomReduceSinkOperators)) { + LOG.info("has job flow correlation"); + correlation.setJobFlowCorrelation(true, bottomReduceSinkOperators); + } + + if (correlation.hasJobFlowCorrelation()) { + corrCtx.addCorrelation(correlation); + } else { + // Since we cannot merge operators into a single MR job from here, + // we should remove ReduceSinkOperators added into walked in exploitJFC + corrCtx.removeWalkedAll(correlation.getAllReduceSinkOperators()); + } + + corrCtx.addWalked(op); + return null; + } + } + + private NodeProcessor getDefaultProc() { + return new NodeProcessor() { + @Override + public Object process(Node nd, Stack stack, + NodeProcessorCtx ctx, Object... nodeOutputs) throws SemanticException { + Operator op = (Operator) nd; + LOG.info("Walk to operator " + op.getIdentifier() + " " + + op.getName() + ". No actual work to do"); + CorrelationNodeProcCtx correlationCtx = (CorrelationNodeProcCtx) ctx; + if (op.getName().equals(MapJoinOperator.getOperatorName())) { + correlationCtx.setAbort(true); + correlationCtx.getAbortReasons().add("Found MAPJOIN"); + } + if (op.getName().equals(FileSinkOperator.getOperatorName())) { + correlationCtx.incrementFileSinkOperatorCount(); + } + return null; + } + }; + } + + protected class CorrelationNodeProcCtx extends AbstractCorrelationProcCtx { + + private boolean abort; + private final List abortReasons; + + private final Set walked; + + private final List correlations; + + private int fileSinkOperatorCount; + + public CorrelationNodeProcCtx(ParseContext pctx) { + super(pctx); + walked = new HashSet(); + correlations = new ArrayList(); + abort = false; + abortReasons = new ArrayList(); + fileSinkOperatorCount = 0; + } + + public void setAbort(boolean abort) { + this.abort = abort; + } + + public boolean isAbort() { + return abort; + } + + public List getAbortReasons() { + return abortReasons; + } + + public void addCorrelation(IntraQueryCorrelation correlation) { + correlations.add(correlation); + } + + public List getCorrelations() { + return correlations; + } + + public boolean isWalked(ReduceSinkOperator op) { + return walked.contains(op); + } + + public void addWalked(ReduceSinkOperator op) { + walked.add(op); + } + + public void addWalkedAll(Collection c) { + walked.addAll(c); + } + + public void removeWalked(ReduceSinkOperator op) { + walked.remove(op); + } + + public void removeWalkedAll(Collection c) { + walked.removeAll(c); + } + + public void incrementFileSinkOperatorCount() { + fileSinkOperatorCount++; + if (fileSinkOperatorCount == 2) { + abort = true; + abortReasons.add( + "-- Currently, a query with multiple FileSinkOperators are not supported."); + } + } + } + +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationUtilities.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationUtilities.java @@ -0,0 +1,492 @@ +/** + * 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.correlation; + + +import java.io.Serializable; +import java.lang.reflect.Array; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.ExtractOperator; +import org.apache.hadoop.hive.ql.exec.FilterOperator; +import org.apache.hadoop.hive.ql.exec.ForwardOperator; +import org.apache.hadoop.hive.ql.exec.GroupByOperator; +import org.apache.hadoop.hive.ql.exec.JoinOperator; +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.RowSchema; +import org.apache.hadoop.hive.ql.exec.ScriptOperator; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.OpParseContext; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.RowResolver; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.AggregationDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; +import org.apache.hadoop.hive.ql.plan.GroupByDesc; +import org.apache.hadoop.hive.ql.plan.JoinCondDesc; +import org.apache.hadoop.hive.ql.plan.JoinDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.SelectDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; + +/** + * Utilities for both CorrelationOptimizer and ReduceSinkDeDuplication. + * + */ +public final class CorrelationUtilities { + + protected static boolean isExisted(ExprNodeDesc expr, List columns) { + for (ExprNodeDesc thisExpr : columns) { + if (thisExpr != null && thisExpr.isSame(expr)) { + return true; + } + } + return false; + } + + protected static String getColumnName( + Map opColumnExprMap, ExprNodeDesc expr) { + for (Entry entry : opColumnExprMap.entrySet()) { + ExprNodeDesc thisExpr = entry.getValue(); + if (thisExpr != null && thisExpr.isSame(expr)) { + return entry.getKey(); + } + } + return null; + } + + protected static boolean hasGroupingSet(ReduceSinkOperator cRS) throws SemanticException { + GroupByOperator cGBYm = getSingleParent(cRS, GroupByOperator.class); + if (cGBYm != null && cGBYm.getConf().isGroupingSetsPresent()) { + return true; + } + return false; + } + + /** + * @param operator the input operator + * @param throwException if throw a exception when the input operator has multiple parents + * @return the single parent or null when the input operator has multiple parents and + * throwException is false; + * @throws HiveException + */ + protected static Operator getSingleParent(Operator operator, + boolean throwException) throws SemanticException { + List> parents = operator.getParentOperators(); + if (parents != null && parents.size() == 1) { + return parents.get(0); + } + if (throwException) { + if (parents == null) { + throw new SemanticException("Operator " + operator.getName() + " (ID: " + + operator.getIdentifier() + ") does not have any parent, but we expect 1 parent."); + } else if (parents.size() > 1) { + throw new SemanticException("Operator " + operator.getName() + " (ID: " + + operator.getIdentifier() + ") has " + parents.size() + + " parents, but we expect 1 parent."); + } + } + return null; + } + + protected static Operator getSingleParent(Operator operator) throws SemanticException { + return getSingleParent(operator, false); + } + + /** + * @param operator the input operator + * @param throwException if throw a exception when the input operator has multiple children + * @return the single child or null when the input operator has multiple children and + * throwException is false; + * @throws HiveException + */ + protected static Operator getSingleChild(Operator operator, + boolean throwException) throws SemanticException { + List> children = operator.getChildOperators(); + if (children != null && children.size() == 1) { + return children.get(0); + } + if (throwException) { + if (children == null) { + throw new SemanticException("Operator " + operator.getName() + " (ID: " + + operator.getIdentifier() + ") does not have any parent, but we expect 1 parent."); + } else if (children.size() > 1) { + throw new SemanticException("Operator " + operator.getName() + " (ID: " + + operator.getIdentifier() + ") has " + children.size() + + " parents, but we expect 1 parent."); + } + } + return null; + } + + protected static Operator getSingleChild(Operator operator) throws SemanticException { + return getSingleChild(operator, false); + } + + protected static T getSingleChild(Operator operator, Class type) + throws SemanticException { + Operator parent = getSingleChild(operator); + return type.isInstance(parent) ? (T)parent : null; + } + + protected static T getSingleParent(Operator operator, Class type) + throws SemanticException { + Operator parent = getSingleParent(operator); + return type.isInstance(parent) ? (T)parent : null; + } + + protected static Operator getStartForGroupBy(ReduceSinkOperator cRS) + throws SemanticException { + Operator parent = getSingleParent(cRS); + return parent instanceof GroupByOperator ? parent : cRS; // skip map-aggr GBY + } + + + protected static boolean[] getSortedTags(JoinOperator joinOp) { + boolean[] result = new boolean[joinOp.getParentOperators().size()]; + for (int tag = 0; tag < result.length; tag++) { + result[tag] = isSortedTag(joinOp, tag); + } + return result; + } + + // for left outer joins, left alias is sorted but right alias might be not + // (nulls, etc.). vice versa. + protected static boolean isSortedTag(JoinOperator joinOp, int tag) { + for (JoinCondDesc cond : joinOp.getConf().getConds()) { + switch (cond.getType()) { + case JoinDesc.LEFT_OUTER_JOIN: + if (cond.getRight() == tag) { + return false; + } + continue; + case JoinDesc.RIGHT_OUTER_JOIN: + if (cond.getLeft() == tag) { + return false; + } + continue; + case JoinDesc.FULL_OUTER_JOIN: + if (cond.getLeft() == tag || cond.getRight() == tag) { + return false; + } + } + } + return true; + } + + protected static int indexOf(ExprNodeDesc cexpr, ExprNodeDesc[] pexprs, Operator child, + Operator[] parents, boolean[] sorted) throws SemanticException { + for (int tag = 0; tag < parents.length; tag++) { + if (sorted[tag] && + pexprs[tag].isSame(ExprNodeDescUtils.backtrack(cexpr, child, parents[tag]))) { + return tag; + } + } + return -1; + } + + protected static > T findPossibleParent(Operator start, Class target, + boolean trustScript) throws SemanticException { + T[] parents = findPossibleParents(start, target, trustScript); + return parents != null && parents.length == 1 ? parents[0] : null; + } + + @SuppressWarnings("unchecked") + protected static > T[] findPossibleParents( + Operator start, Class target, + boolean trustScript) throws SemanticException { + Operator cursor = getSingleParent(start); + for (; cursor != null; cursor = getSingleParent(cursor)) { + if (target.isAssignableFrom(cursor.getClass())) { + T[] array = (T[]) Array.newInstance(target, 1); + array[0] = (T) cursor; + return array; + } + if (cursor instanceof JoinOperator) { + return findParents((JoinOperator) cursor, target); + } + if (cursor instanceof ScriptOperator && !trustScript) { + return null; + } + if (!(cursor instanceof SelectOperator + || cursor instanceof FilterOperator + || cursor instanceof ExtractOperator + || cursor instanceof ForwardOperator + || cursor instanceof ScriptOperator + || cursor instanceof ReduceSinkOperator)) { + return null; + } + } + return null; + } + + @SuppressWarnings("unchecked") + protected static > T[] findParents(JoinOperator join, Class target) + throws SemanticException { + List> parents = join.getParentOperators(); + T[] result = (T[]) Array.newInstance(target, parents.size()); + for (int tag = 0; tag < result.length; tag++) { + Operator cursor = parents.get(tag); + for (; cursor != null; cursor = getSingleParent(cursor)) { + if (target.isAssignableFrom(cursor.getClass())) { + result[tag] = (T) cursor; + break; + } + } + if (result[tag] == null) { + throw new IllegalStateException("failed to find " + target.getSimpleName() + + " from " + join + " on tag " + tag); + } + } + return result; + } + + /** + * Search the query plan tree from startPoint to the bottom. If there is no ReduceSinkOperator + * between startPoint and the corresponding TableScanOperator, return the corresponding + * TableScanOperator. Otherwise, return null. + * @param startPoint the operator which the search will start at + * @return the TableScanOperator traced from startPoint. Null, if the search encounters any + * ReduceSinkOperator. + */ + protected static TableScanOperator findTableScanOperator( + Operator startPoint) { + Operator thisOp = startPoint.getParentOperators().get(0); + while (true) { + if (thisOp.getName().equals(ReduceSinkOperator.getOperatorName())) { + return null; + } else if (thisOp.getName().equals(TableScanOperator.getOperatorName())) { + return (TableScanOperator) thisOp; + } else { + if (thisOp.getParentOperators() != null) { + thisOp = thisOp.getParentOperators().get(0); + } else { + break; + } + } + } + return null; + } + + /** + * Find all sibling ReduceSinkOperators (which have the same child operator of op) of op (op + * included). + * @throws SemanticException + */ + public static List findSiblingReduceSinkOperators(ReduceSinkOperator op) + throws SemanticException { + List siblingRSs = new ArrayList(); + Operator child = getSingleChild(op, true); + for (Operator parent: child.getParentOperators()) { + if (parent instanceof ReduceSinkOperator) { + siblingRSs.add((ReduceSinkOperator)parent); + } else { + throw new SemanticException("An sibling of a ReduceSinkOperatpr is not a" + + "ReduceSinkOperatpr."); + } + } + return siblingRSs; + } + + /** + * Find all sibling operators (which have the same child operator of op) of op (op + * included). + * @throws SemanticException + */ + public static List> findSiblingOperators( + Operator op) + throws SemanticException { + Operator child = getSingleChild(op, true); + return child.getParentOperators(); + } + + protected static SelectOperator replaceReduceSinkWithSelectOperator(ReduceSinkOperator childRS, + ParseContext context, AbstractCorrelationProcCtx procCtx) throws SemanticException { + SelectOperator select = replaceOperatorWithSelect(childRS, context, procCtx); + select.getConf().setOutputColumnNames(childRS.getConf().getOutputValueColumnNames()); + select.getConf().setColList(childRS.getConf().getValueCols()); + return select; + } + + // replace the cRS to SEL operator + // If child if cRS is EXT, EXT also should be removed + protected static SelectOperator replaceOperatorWithSelect(Operator operator, + ParseContext context, AbstractCorrelationProcCtx procCtx) + throws SemanticException { + RowResolver inputRR = context.getOpParseCtx().get(operator).getRowResolver(); + SelectDesc select = new SelectDesc(null, null); + + Operator parent = getSingleParent(operator); + Operator child = getSingleChild(operator); + + parent.getChildOperators().clear(); + + SelectOperator sel = (SelectOperator) putOpInsertMap( + OperatorFactory.getAndMakeChild(select, new RowSchema(inputRR + .getColumnInfos()), parent), inputRR, context); + + sel.setColumnExprMap(operator.getColumnExprMap()); + + sel.setChildOperators(operator.getChildOperators()); + for (Operator ch : operator.getChildOperators()) { + ch.replaceParent(operator, sel); + } + if (child instanceof ExtractOperator) { + removeOperator(child, getSingleChild(child), sel, context); + procCtx.addRemovedOperator(child); + } + operator.setChildOperators(null); + operator.setParentOperators(null); + procCtx.addRemovedOperator(operator); + return sel; + } + + protected static void removeReduceSinkForGroupBy(ReduceSinkOperator cRS, GroupByOperator cGBYr, + ParseContext context, AbstractCorrelationProcCtx procCtx) throws SemanticException { + + Operator parent = getSingleParent(cRS); + + if (parent instanceof GroupByOperator) { + // pRS-cGBYm-cRS-cGBYr (map aggregation) --> pRS-cGBYr(COMPLETE) + // copies desc of cGBYm to cGBYr and remove cGBYm and cRS + GroupByOperator cGBYm = (GroupByOperator) parent; + + cGBYr.getConf().setKeys(cGBYm.getConf().getKeys()); + cGBYr.getConf().setAggregators(cGBYm.getConf().getAggregators()); + for (AggregationDesc aggr : cGBYm.getConf().getAggregators()) { + aggr.setMode(GenericUDAFEvaluator.Mode.COMPLETE); + } + cGBYr.setColumnExprMap(cGBYm.getColumnExprMap()); + cGBYr.setSchema(cGBYm.getSchema()); + RowResolver resolver = context.getOpParseCtx().get(cGBYm).getRowResolver(); + context.getOpParseCtx().get(cGBYr).setRowResolver(resolver); + } else { + // pRS-cRS-cGBYr (no map aggregation) --> pRS-cGBYr(COMPLETE) + // revert expressions of cGBYr to that of cRS + cGBYr.getConf().setKeys(ExprNodeDescUtils.backtrack(cGBYr.getConf().getKeys(), cGBYr, cRS)); + for (AggregationDesc aggr : cGBYr.getConf().getAggregators()) { + aggr.setParameters(ExprNodeDescUtils.backtrack(aggr.getParameters(), cGBYr, cRS)); + } + + Map oldMap = cGBYr.getColumnExprMap(); + RowResolver oldRR = context.getOpParseCtx().get(cGBYr).getRowResolver(); + + Map newMap = new HashMap(); + RowResolver newRR = new RowResolver(); + + List outputCols = cGBYr.getConf().getOutputColumnNames(); + for (int i = 0; i < outputCols.size(); i++) { + String colName = outputCols.get(i); + String[] nm = oldRR.reverseLookup(colName); + ColumnInfo colInfo = oldRR.get(nm[0], nm[1]); + newRR.put(nm[0], nm[1], colInfo); + ExprNodeDesc colExpr = ExprNodeDescUtils.backtrack(oldMap.get(colName), cGBYr, cRS); + if (colExpr != null) { + newMap.put(colInfo.getInternalName(), colExpr); + } + } + cGBYr.setColumnExprMap(newMap); + cGBYr.setSchema(new RowSchema(newRR.getColumnInfos())); + context.getOpParseCtx().get(cGBYr).setRowResolver(newRR); + } + cGBYr.getConf().setMode(GroupByDesc.Mode.COMPLETE); + + removeOperator(cRS, cGBYr, parent, context); + procCtx.addRemovedOperator(cRS); + + if (parent instanceof GroupByOperator) { + removeOperator(parent, cGBYr, getSingleParent(parent), context); + procCtx.addRemovedOperator(cGBYr); + } + } + + /** throw a exception if the input operator is null + * @param operator + * @throws HiveException + */ + protected static void isNullOperator(Operator operator) throws SemanticException { + if (operator == null) { + throw new SemanticException("Operator " + operator.getName() + " (ID: " + + operator.getIdentifier() + ") is null."); + } + } + + /** + * @param newOperator the operator will be inserted between child and parent + * @param child + * @param parent + * @param context + * @throws HiveException + */ + protected static void insertOperatorBetween( + Operator newOperator, Operator parent, Operator child) + throws SemanticException { + isNullOperator(newOperator); + isNullOperator(parent); + isNullOperator(child); + + if (parent != getSingleParent(child)) { + throw new SemanticException("Operator " + parent.getName() + " (ID: " + + parent.getIdentifier() + ") is not the only parent of Operator " + + child.getName() + " (ID: " + child.getIdentifier() + ")"); + } + if (child != getSingleChild(parent)) { + throw new SemanticException("Operator " + child.getName() + " (ID: " + + child.getIdentifier() + ") is not the only child of Operator " + + parent.getName() + " (ID: " + parent.getIdentifier() + ")"); + } + + newOperator.setParentOperators(Utilities.makeList(parent)); + newOperator.setChildOperators(Utilities.makeList(child)); + + child.setParentOperators(Utilities.makeList(newOperator)); + parent.setChildOperators(Utilities.makeList(newOperator)); + } + + protected static void removeOperator(Operator target, Operator child, Operator parent, + ParseContext context) { + for (Operator aparent : target.getParentOperators()) { + aparent.replaceChild(target, child); + } + for (Operator achild : target.getChildOperators()) { + achild.replaceParent(target, parent); + } + target.setChildOperators(null); + target.setParentOperators(null); + context.getOpParseCtx().remove(target); + } + + protected static Operator putOpInsertMap(Operator op, RowResolver rr, + ParseContext context) { + OpParseContext ctx = new OpParseContext(rr); + context.getOpParseCtx().put(op, ctx); + return op; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/IntraQueryCorrelation.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/IntraQueryCorrelation.java @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.optimizer.correlation; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; + +/** + * IntraQueryCorrelation records a sub-tree of the query plan tree which can be + * evaluated in a single MR job. The boundary of this sub-tree is recorded by + * the ReduceSinkOperators the the bottom of this sub-tree. + * Also, allReduceSinkOperators in IntraQueryCorrelation contains all + * ReduceSinkOperators of this sub-tree. + */ +public class IntraQueryCorrelation { + private boolean jobFlowCorrelation; + + // The bottom layer ReduceSinkOperators. These ReduceSinkOperators are used + // to record the boundary of this sub-tree which can be evaluated in a single MR + // job. + private List bottomReduceSinkOperators; + + // The number of reducer(s) should be used for those bottom layer ReduceSinkOperators + private int numReducers; + // This is the min number of reducer(s) for the bottom layer ReduceSinkOperators to avoid query + // executed on too small number of reducers. + private final int minReducers; + + // All ReduceSinkOperators in this sub-tree. This set is used when we start to remove unnecessary + // ReduceSinkOperators. + private final Set allReduceSinkOperators; + + // Since we merge multiple operation paths, we assign new tags to bottom layer + // ReduceSinkOperatos. This mapping is used to map new tags to original tags associated + // to these bottom layer ReduceSinkOperators. + private final Map newTagToOldTag; + + // A map from new tags to indices of children of DemuxOperator (the first Operator at the + // Reduce side of optimized plan) + private final Map newTagToChildIndex; + + public IntraQueryCorrelation(int minReducers) { + this.jobFlowCorrelation = false; + this.numReducers = -1; + this.minReducers = minReducers; + this.allReduceSinkOperators = new HashSet(); + this.newTagToOldTag = new HashMap(); + this.newTagToChildIndex = new HashMap(); + } + + public Map getNewTagToOldTag() { + return newTagToOldTag; + } + + public Map getNewTagToChildIndex() { + return newTagToChildIndex; + } + + public void setNewTag(Integer newTag, Integer oldTag, Integer childIndex) { + newTagToOldTag.put(newTag, oldTag); + newTagToChildIndex.put(newTag, childIndex); + } + public void addToAllReduceSinkOperators(ReduceSinkOperator rsop) { + allReduceSinkOperators.add(rsop); + } + + public Set getAllReduceSinkOperators() { + return allReduceSinkOperators; + } + + public void setJobFlowCorrelation(boolean jobFlowCorrelation, + List bottomReduceSinkOperators) { + this.jobFlowCorrelation = jobFlowCorrelation; + this.bottomReduceSinkOperators = bottomReduceSinkOperators; + } + + public boolean hasJobFlowCorrelation() { + return jobFlowCorrelation; + } + + public List getBottomReduceSinkOperators() { + return bottomReduceSinkOperators; + } + + public int getNumReducers() { + return numReducers; + } + + public boolean adjustNumReducers(int newNumReducers) { + assert newNumReducers != 0; + if (newNumReducers > 0) { + // If the new numReducer is less than minReducer, we will not consider + // ReduceSinkOperator with this newNumReducer as a correlated ReduceSinkOperator + if (newNumReducers < minReducers) { + return false; + } + if (numReducers > 0) { + if (newNumReducers != numReducers) { + // If (numReducers > 0 && newNumReducers > 0 && newNumReducers != numReducers), + // we will not consider ReduceSinkOperator with this newNumReducer as a correlated + // ReduceSinkOperator + return false; + } + } else { + // if numReducers < 0 and newNumReducers > 0 + numReducers = newNumReducers; + } + } + + return true; + } + +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/QueryPlanTreeTransformation.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/QueryPlanTreeTransformation.java @@ -0,0 +1,259 @@ +/** + * 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.correlation; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.DemuxOperator; +import org.apache.hadoop.hive.ql.exec.GroupByOperator; +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.Utilities; +import org.apache.hadoop.hive.ql.optimizer.correlation.CorrelationOptimizer.CorrelationNodeProcCtx; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.DemuxDesc; +import org.apache.hadoop.hive.ql.plan.MuxDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; + +/** + * QueryPlanTreeTransformation contains static methods used to transform + * the query plan tree (operator tree) based on the correlation we have + * detected by Correlation Optimizer. + */ +public class QueryPlanTreeTransformation { + private static final Log LOG = LogFactory.getLog(QueryPlanTreeTransformation.class.getName()); + + private static void setNewTag(IntraQueryCorrelation correlation, + List> childrenOfDemux, + ReduceSinkOperator rsop, Map bottomRSToNewTag) + throws SemanticException { + int newTag = bottomRSToNewTag.get(rsop); + int oldTag = rsop.getConf().getTag(); + // if this child of dispatcher does not use tag, we just set the oldTag to 0; + if (oldTag == -1) { + oldTag = 0; + } + Operator child = CorrelationUtilities.getSingleChild(rsop, true); + if (!childrenOfDemux.contains(child)) { + childrenOfDemux.add(child); + } + int childIndex = childrenOfDemux.indexOf(child); + correlation.setNewTag(newTag, oldTag, childIndex); + rsop.getConf().setTag(newTag); + } + + /** Based on the correlation, we transform the query plan tree (operator tree). + * In here, we first create DemuxOperator and all bottom ReduceSinkOperators + * (bottom means near TableScanOperaotr) in the correlation will be be + * the parents of the DemuxOperaotr. We also reassign tags to those + * ReduceSinkOperators. Then, we use MuxOperators to replace ReduceSinkOperators + * which are not bottom ones in this correlation. + * Example: The original operator tree is ... + * JOIN2 + * / \ + * RS4 RS5 + * / \ + * GBY1 JOIN1 + * | / \ + * RS1 RS2 RS3 + * If GBY1, JOIN1, and JOIN2 can be executed in the same reducer + * (optimized by Correlation Optimizer). + * The new operator tree will be ... + * JOIN2 + * | + * MUX + * / \ + * GBY1 JOIN1 + * \ / + * DEMUX + * / | \ + * / | \ + * / | \ + * RS1 RS2 RS3 + * @param pCtx + * @param corrCtx + * @param correlation + * @throws SemanticException + */ + protected static void applyCorrelation( + ParseContext pCtx, + CorrelationNodeProcCtx corrCtx, + IntraQueryCorrelation correlation) + throws SemanticException { + + final List bottomReduceSinkOperators = + correlation.getBottomReduceSinkOperators(); + final int numReducers = correlation.getNumReducers(); + List> childrenOfDemux = + new ArrayList>(); + List> parentRSsOfDemux = + new ArrayList>(); + Map childIndexToOriginalNumParents = + new HashMap(); + List keysSerializeInfos = new ArrayList(); + List valuessSerializeInfos = new ArrayList(); + Map bottomRSToNewTag = + new HashMap(); + int newTag = 0; + for (ReduceSinkOperator rsop: bottomReduceSinkOperators) { + rsop.getConf().setNumReducers(numReducers); + bottomRSToNewTag.put(rsop, newTag); + parentRSsOfDemux.add(rsop); + keysSerializeInfos.add(rsop.getConf().getKeySerializeInfo()); + valuessSerializeInfos.add(rsop.getConf().getValueSerializeInfo()); + Operator child = CorrelationUtilities.getSingleChild(rsop, true); + if (!childrenOfDemux.contains(child)) { + childrenOfDemux.add(child); + int childIndex = childrenOfDemux.size() - 1; + childIndexToOriginalNumParents.put(childIndex, child.getNumParent()); + } + newTag++; + } + + for (ReduceSinkOperator rsop: bottomReduceSinkOperators) { + setNewTag(correlation, childrenOfDemux, rsop, bottomRSToNewTag); + } + + // Create the DemuxOperaotr + DemuxDesc demuxDesc = + new DemuxDesc( + correlation.getNewTagToOldTag(), + correlation.getNewTagToChildIndex(), + childIndexToOriginalNumParents, + keysSerializeInfos, + valuessSerializeInfos); + Operator demuxOp = OperatorFactory.get(demuxDesc); + demuxOp.setChildOperators(childrenOfDemux); + demuxOp.setParentOperators(parentRSsOfDemux); + for (Operator child: childrenOfDemux) { + List> parentsWithMultipleDemux = + new ArrayList>(); + boolean hasBottomReduceSinkOperators = false; + boolean hasNonBottomReduceSinkOperators = false; + for (int i = 0; i < child.getParentOperators().size(); i++) { + Operator p = child.getParentOperators().get(i); + assert p instanceof ReduceSinkOperator; + ReduceSinkOperator rsop = (ReduceSinkOperator)p; + if (bottomReduceSinkOperators.contains(rsop)) { + hasBottomReduceSinkOperators = true; + parentsWithMultipleDemux.add(demuxOp); + } else { + hasNonBottomReduceSinkOperators = true; + parentsWithMultipleDemux.add(rsop); + } + } + if (hasBottomReduceSinkOperators && hasNonBottomReduceSinkOperators) { + child.setParentOperators(parentsWithMultipleDemux); + } else { + child.setParentOperators(Utilities.makeList(demuxOp)); + } + } + for (Operator parent: parentRSsOfDemux) { + parent.setChildOperators(Utilities.makeList(demuxOp)); + } + + // replace all ReduceSinkOperators which are not at the bottom of + // this correlation to MuxOperators + Set handledRSs = new HashSet(); + for (ReduceSinkOperator rsop : correlation.getAllReduceSinkOperators()) { + if (!bottomReduceSinkOperators.contains(rsop)) { + if (handledRSs.contains(rsop)) { + continue; + } + Operator childOP = + CorrelationUtilities.getSingleChild(rsop, true); + if (childOP instanceof GroupByOperator) { + CorrelationUtilities.removeReduceSinkForGroupBy( + rsop, (GroupByOperator)childOP, pCtx, corrCtx); + List> parentsOfMux = + new ArrayList>(); + Operator parentOp = + CorrelationUtilities.getSingleParent(childOP, true); + parentsOfMux.add(parentOp); + Operator mux = OperatorFactory.get( + new MuxDesc(parentsOfMux)); + mux.setChildOperators(Utilities.makeList(childOP)); + mux.setParentOperators(parentsOfMux); + childOP.setParentOperators(Utilities.makeList(mux)); + parentOp.setChildOperators(Utilities.makeList(mux)); + } else { + // childOp is a JoinOperator + List> parentsOfMux = + new ArrayList>(); + List> siblingOPs = + CorrelationUtilities.findSiblingOperators(rsop); + for (Operator op: siblingOPs) { + if (op instanceof DemuxOperator) { + parentsOfMux.add(op); + } else if (op instanceof ReduceSinkOperator){ + GroupByOperator pGBYm = + CorrelationUtilities.getSingleParent(op, GroupByOperator.class); + if (pGBYm != null) { + // We get a semi join at here. + // This map-side GroupByOperator needs to be removed + CorrelationUtilities.removeOperator( + pGBYm, op, CorrelationUtilities.getSingleParent(pGBYm, true), pCtx); + } + handledRSs.add((ReduceSinkOperator)op); + parentsOfMux.add(CorrelationUtilities.getSingleParent(op, true)); + } else { + throw new SemanticException("An slibing of ReduceSinkOperator is nethier a " + + "DemuxOperator nor a ReduceSinkOperator"); + } + } + MuxDesc muxDesc = new MuxDesc(siblingOPs); + Operator mux = OperatorFactory.get(muxDesc); + mux.setChildOperators(Utilities.makeList(childOP)); + mux.setParentOperators(parentsOfMux); + + for (Operator op: parentsOfMux) { + if (op instanceof DemuxOperator) { + // op is a DemuxOperator and it directly connects to childOP. + // We will add this MuxOperator between DemuxOperator + // and childOP. + if (op.getChildOperators().contains(childOP)) { + op.replaceChild(childOP, mux); + } + } else { + // op is not a DemuxOperator, so it should have + // a single child. + op.setChildOperators(Utilities.makeList(mux)); + } + } + childOP.setParentOperators(Utilities.makeList(mux)); + } + } + } + for (ReduceSinkOperator rsop: handledRSs) { + rsop.setChildOperators(null); + rsop.setParentOperators(null); + pCtx.getOpParseCtx().remove(rsop); + } + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java @@ -0,0 +1,505 @@ +/** + * 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.correlation; + +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVECONVERTJOIN; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASK; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; + +import org.apache.hadoop.hive.ql.exec.ExtractOperator; +import org.apache.hadoop.hive.ql.exec.GroupByOperator; +import org.apache.hadoop.hive.ql.exec.JoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.lib.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.optimizer.Transform; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; +import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; + +/** + * If two reducer sink operators share the same partition/sort columns and order, + * they can be merged. This should happen after map join optimization because map + * join optimization will remove reduce sink operators. + * + * This optimizer removes/replaces child-RS (not parent) which is safer way for DefaultGraphWalker. + */ +public class ReduceSinkDeDuplication implements Transform { + + private static final String RS = ReduceSinkOperator.getOperatorName(); + private static final String GBY = GroupByOperator.getOperatorName(); + private static final String JOIN = JoinOperator.getOperatorName(); + + protected ParseContext pGraphContext; + + @Override + public ParseContext transform(ParseContext pctx) throws SemanticException { + pGraphContext = pctx; + + // generate pruned column list for all relevant operators + ReduceSinkDeduplicateProcCtx cppCtx = new ReduceSinkDeduplicateProcCtx(pGraphContext); + + // for auto convert map-joins, it not safe to dedup in here (todo) + boolean mergeJoins = !pctx.getConf().getBoolVar(HIVECONVERTJOIN) && + !pctx.getConf().getBoolVar(HIVECONVERTJOINNOCONDITIONALTASK); + + // If multiple rules can be matched with same cost, last rule will be choosen as a processor + // see DefaultRuleDispatcher#dispatch() + Map opRules = new LinkedHashMap(); + opRules.put(new RuleRegExp("R1", RS + "%.*%" + RS + "%"), + ReduceSinkDeduplicateProcFactory.getReducerReducerProc()); + opRules.put(new RuleRegExp("R2", RS + "%" + GBY + "%.*%" + RS + "%"), + ReduceSinkDeduplicateProcFactory.getGroupbyReducerProc()); + if (mergeJoins) { + opRules.put(new RuleRegExp("R3", JOIN + "%.*%" + RS + "%"), + ReduceSinkDeduplicateProcFactory.getJoinReducerProc()); + } + // TODO RS+JOIN + + // The dispatcher fires the processor corresponding to the closest matching + // rule and passes the context along + Dispatcher disp = new DefaultRuleDispatcher(ReduceSinkDeduplicateProcFactory + .getDefaultProc(), opRules, cppCtx); + GraphWalker ogw = new DefaultGraphWalker(disp); + + // Create a list of topop nodes + ArrayList topNodes = new ArrayList(); + topNodes.addAll(pGraphContext.getTopOps().values()); + ogw.startWalking(topNodes, null); + return pGraphContext; + } + + protected class ReduceSinkDeduplicateProcCtx extends AbstractCorrelationProcCtx { + + public ReduceSinkDeduplicateProcCtx(ParseContext pctx) { + super(pctx); + } + } + + static class ReduceSinkDeduplicateProcFactory { + + public static NodeProcessor getReducerReducerProc() { + return new ReducerReducerProc(); + } + + public static NodeProcessor getGroupbyReducerProc() { + return new GroupbyReducerProc(); + } + + public static NodeProcessor getJoinReducerProc() { + return new JoinReducerProc(); + } + + public static NodeProcessor getDefaultProc() { + return new DefaultProc(); + } + } + + /* + * do nothing. + */ + static class DefaultProc implements NodeProcessor { + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + return null; + } + } + + public abstract static class AbsctractReducerReducerProc implements NodeProcessor { + + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + ReduceSinkDeduplicateProcCtx dedupCtx = (ReduceSinkDeduplicateProcCtx) procCtx; + if (dedupCtx.hasBeenRemoved((Operator) nd)) { + return false; + } + ReduceSinkOperator cRS = (ReduceSinkOperator) nd; + Operator child = CorrelationUtilities.getSingleChild(cRS); + if (child instanceof JoinOperator) { + return false; // not supported + } + if (child instanceof GroupByOperator) { + GroupByOperator cGBY = (GroupByOperator) child; + if (!CorrelationUtilities.hasGroupingSet(cRS) && !cGBY.getConf().isGroupingSetsPresent()) { + return process(cRS, cGBY, dedupCtx); + } + return false; + } + if (child instanceof ExtractOperator) { + return process(cRS, dedupCtx); + } + return false; + } + + protected abstract Object process(ReduceSinkOperator cRS, ReduceSinkDeduplicateProcCtx dedupCtx) + throws SemanticException; + + protected abstract Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, + ReduceSinkDeduplicateProcCtx dedupCtx) throws SemanticException; + + // for JOIN-RS case, it's not possible generally to merge if child has + // more key/partition columns than parents + protected boolean merge(ReduceSinkOperator cRS, JoinOperator pJoin, int minReducer) + throws SemanticException { + List> parents = pJoin.getParentOperators(); + ReduceSinkOperator[] pRSs = parents.toArray(new ReduceSinkOperator[parents.size()]); + ReduceSinkDesc cRSc = cRS.getConf(); + ReduceSinkDesc pRS0c = pRSs[0].getConf(); + if (cRSc.getKeyCols().size() > pRS0c.getKeyCols().size()) { + return false; + } + if (cRSc.getPartitionCols().size() > pRS0c.getPartitionCols().size()) { + return false; + } + Integer moveReducerNumTo = checkNumReducer(cRSc.getNumReducers(), pRS0c.getNumReducers()); + if (moveReducerNumTo == null || + moveReducerNumTo > 0 && cRSc.getNumReducers() < minReducer) { + return false; + } + + Integer moveRSOrderTo = checkOrder(cRSc.getOrder(), pRS0c.getOrder()); + if (moveRSOrderTo == null) { + return false; + } + + boolean[] sorted = CorrelationUtilities.getSortedTags(pJoin); + + int cKeySize = cRSc.getKeyCols().size(); + for (int i = 0; i < cKeySize; i++) { + ExprNodeDesc cexpr = cRSc.getKeyCols().get(i); + ExprNodeDesc[] pexprs = new ExprNodeDesc[pRSs.length]; + for (int tag = 0; tag < pRSs.length; tag++) { + pexprs[tag] = pRSs[tag].getConf().getKeyCols().get(i); + } + int found = CorrelationUtilities.indexOf(cexpr, pexprs, cRS, pRSs, sorted); + if (found < 0) { + return false; + } + } + int cPartSize = cRSc.getPartitionCols().size(); + for (int i = 0; i < cPartSize; i++) { + ExprNodeDesc cexpr = cRSc.getPartitionCols().get(i); + ExprNodeDesc[] pexprs = new ExprNodeDesc[pRSs.length]; + for (int tag = 0; tag < pRSs.length; tag++) { + pexprs[tag] = pRSs[tag].getConf().getPartitionCols().get(i); + } + int found = CorrelationUtilities.indexOf(cexpr, pexprs, cRS, pRSs, sorted); + if (found < 0) { + return false; + } + } + + if (moveReducerNumTo > 0) { + for (ReduceSinkOperator pRS : pRSs) { + pRS.getConf().setNumReducers(cRS.getConf().getNumReducers()); + } + } + return true; + } + + /** + * Current RSDedup remove/replace child RS. So always copies + * more specific part of configurations of child RS to that of parent RS. + */ + protected boolean merge(ReduceSinkOperator cRS, ReduceSinkOperator pRS, int minReducer) + throws SemanticException { + int[] result = checkStatus(cRS, pRS, minReducer); + if (result == null) { + return false; + } + if (result[0] > 0) { + ArrayList childKCs = cRS.getConf().getKeyCols(); + pRS.getConf().setKeyCols(ExprNodeDescUtils.backtrack(childKCs, cRS, pRS)); + } + if (result[1] > 0) { + ArrayList childPCs = cRS.getConf().getPartitionCols(); + pRS.getConf().setPartitionCols(ExprNodeDescUtils.backtrack(childPCs, cRS, pRS)); + } + if (result[2] > 0) { + pRS.getConf().setOrder(cRS.getConf().getOrder()); + } + if (result[3] > 0) { + pRS.getConf().setNumReducers(cRS.getConf().getNumReducers()); + } + return true; + } + + /** + * Returns merge directions between two RSs for criterias (ordering, number of reducers, + * reducer keys, partition keys). Returns null if any of categories is not mergeable. + * + * Values for each index can be -1, 0, 1 + * 1. 0 means two configuration in the category is the same + * 2. for -1, configuration of parent RS is more specific than child RS + * 3. for 1, configuration of child RS is more specific than parent RS + */ + private int[] checkStatus(ReduceSinkOperator cRS, ReduceSinkOperator pRS, int minReducer) + throws SemanticException { + ReduceSinkDesc cConf = cRS.getConf(); + ReduceSinkDesc pConf = pRS.getConf(); + Integer moveRSOrderTo = checkOrder(cConf.getOrder(), pConf.getOrder()); + if (moveRSOrderTo == null) { + return null; + } + Integer moveReducerNumTo = checkNumReducer(cConf.getNumReducers(), pConf.getNumReducers()); + if (moveReducerNumTo == null || + moveReducerNumTo > 0 && cConf.getNumReducers() < minReducer) { + return null; + } + List ckeys = cConf.getKeyCols(); + List pkeys = pConf.getKeyCols(); + Integer moveKeyColTo = checkExprs(ckeys, pkeys, cRS, pRS); + if (moveKeyColTo == null) { + return null; + } + List cpars = cConf.getPartitionCols(); + List ppars = pConf.getPartitionCols(); + Integer movePartitionColTo = checkExprs(cpars, ppars, cRS, pRS); + if (movePartitionColTo == null) { + return null; + } + return new int[] {moveKeyColTo, movePartitionColTo, moveRSOrderTo, moveReducerNumTo}; + } + + /** + * Overlapping part of keys should be the same between parent and child. + * And if child has more keys than parent, non-overlapping part of keys + * should be backtrackable to parent. + */ + private Integer checkExprs(List ckeys, List pkeys, + ReduceSinkOperator cRS, ReduceSinkOperator pRS) throws SemanticException { + Integer moveKeyColTo = 0; + if (ckeys == null || ckeys.isEmpty()) { + if (pkeys != null && !pkeys.isEmpty()) { + moveKeyColTo = -1; + } + } else { + if (pkeys == null || pkeys.isEmpty()) { + for (ExprNodeDesc ckey : ckeys) { + if (ExprNodeDescUtils.backtrack(ckey, cRS, pRS) == null) { + // cKey is not present in parent + return null; + } + } + moveKeyColTo = 1; + } else { + moveKeyColTo = sameKeys(ckeys, pkeys, cRS, pRS); + } + } + return moveKeyColTo; + } + + // backtrack key exprs of child to parent and compare it with parent's + protected Integer sameKeys(List cexprs, List pexprs, + Operator child, Operator parent) throws SemanticException { + int common = Math.min(cexprs.size(), pexprs.size()); + int limit = Math.max(cexprs.size(), pexprs.size()); + int i = 0; + for (; i < common; i++) { + ExprNodeDesc pexpr = pexprs.get(i); + ExprNodeDesc cexpr = ExprNodeDescUtils.backtrack(cexprs.get(i), child, parent); + if (cexpr == null || !pexpr.isSame(cexpr)) { + return null; + } + } + for (; i < limit; i++) { + if (cexprs.size() > pexprs.size()) { + if (ExprNodeDescUtils.backtrack(cexprs.get(i), child, parent) == null) { + // cKey is not present in parent + return null; + } + } + } + return Integer.valueOf(cexprs.size()).compareTo(pexprs.size()); + } + + // order of overlapping keys should be exactly the same + protected Integer checkOrder(String corder, String porder) { + if (corder == null || corder.trim().equals("")) { + if (porder == null || porder.trim().equals("")) { + return 0; + } + return -1; + } + if (porder == null || porder.trim().equals("")) { + return 1; + } + corder = corder.trim(); + porder = porder.trim(); + int target = Math.min(corder.length(), porder.length()); + if (!corder.substring(0, target).equals(porder.substring(0, target))) { + return null; + } + return Integer.valueOf(corder.length()).compareTo(porder.length()); + } + + /** + * If number of reducers for RS is -1, the RS can have any number of reducers. + * It's generally true except for order-by or forced bucketing cases. + * if both of num-reducers are not -1, those number should be the same. + */ + protected Integer checkNumReducer(int creduce, int preduce) { + if (creduce < 0) { + if (preduce < 0) { + return 0; + } + return -1; + } + if (preduce < 0) { + return 1; + } + if (creduce != preduce) { + return null; + } + return 0; + } + } + + static class GroupbyReducerProc extends AbsctractReducerReducerProc { + + // pRS-pGBY-cRS + @Override + public Object process(ReduceSinkOperator cRS, ReduceSinkDeduplicateProcCtx dedupCtx) + throws SemanticException { + GroupByOperator pGBY = + CorrelationUtilities.findPossibleParent( + cRS, GroupByOperator.class, dedupCtx.trustScript()); + if (pGBY == null) { + return false; + } + ReduceSinkOperator pRS = + CorrelationUtilities.findPossibleParent( + pGBY, ReduceSinkOperator.class, dedupCtx.trustScript()); + if (pRS != null && merge(cRS, pRS, dedupCtx.minReducer())) { + CorrelationUtilities.replaceReduceSinkWithSelectOperator( + cRS, dedupCtx.getPctx(), dedupCtx); + return true; + } + return false; + } + + // pRS-pGBY-cRS-cGBY + @Override + public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, + ReduceSinkDeduplicateProcCtx dedupCtx) + throws SemanticException { + Operator start = CorrelationUtilities.getStartForGroupBy(cRS); + GroupByOperator pGBY = + CorrelationUtilities.findPossibleParent( + start, GroupByOperator.class, dedupCtx.trustScript()); + if (pGBY == null) { + return false; + } + ReduceSinkOperator pRS = + CorrelationUtilities.getSingleParent(pGBY, ReduceSinkOperator.class); + if (pRS != null && merge(cRS, pRS, dedupCtx.minReducer())) { + CorrelationUtilities.removeReduceSinkForGroupBy( + cRS, cGBY, dedupCtx.getPctx(), dedupCtx); + return true; + } + return false; + } + } + + static class JoinReducerProc extends AbsctractReducerReducerProc { + + // pRS-pJOIN-cRS + @Override + public Object process(ReduceSinkOperator cRS, ReduceSinkDeduplicateProcCtx dedupCtx) + throws SemanticException { + JoinOperator pJoin = + CorrelationUtilities.findPossibleParent(cRS, JoinOperator.class, dedupCtx.trustScript()); + if (pJoin != null && merge(cRS, pJoin, dedupCtx.minReducer())) { + pJoin.getConf().setFixedAsSorted(true); + CorrelationUtilities.replaceReduceSinkWithSelectOperator( + cRS, dedupCtx.getPctx(), dedupCtx); + return true; + } + return false; + } + + // pRS-pJOIN-cRS-cGBY + @Override + public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, + ReduceSinkDeduplicateProcCtx dedupCtx) + throws SemanticException { + Operator start = CorrelationUtilities.getStartForGroupBy(cRS); + JoinOperator pJoin = + CorrelationUtilities.findPossibleParent( + start, JoinOperator.class, dedupCtx.trustScript()); + if (pJoin != null && merge(cRS, pJoin, dedupCtx.minReducer())) { + pJoin.getConf().setFixedAsSorted(true); + CorrelationUtilities.removeReduceSinkForGroupBy( + cRS, cGBY, dedupCtx.getPctx(), dedupCtx); + return true; + } + return false; + } + } + + static class ReducerReducerProc extends AbsctractReducerReducerProc { + + // pRS-cRS + @Override + public Object process(ReduceSinkOperator cRS, ReduceSinkDeduplicateProcCtx dedupCtx) + throws SemanticException { + ReduceSinkOperator pRS = + CorrelationUtilities.findPossibleParent( + cRS, ReduceSinkOperator.class, dedupCtx.trustScript()); + if (pRS != null && merge(cRS, pRS, dedupCtx.minReducer())) { + CorrelationUtilities.replaceReduceSinkWithSelectOperator( + cRS, dedupCtx.getPctx(), dedupCtx); + return true; + } + return false; + } + + // pRS-cRS-cGBY + @Override + public Object process(ReduceSinkOperator cRS, GroupByOperator cGBY, + ReduceSinkDeduplicateProcCtx dedupCtx) + throws SemanticException { + Operator start = CorrelationUtilities.getStartForGroupBy(cRS); + ReduceSinkOperator pRS = + CorrelationUtilities.findPossibleParent( + start, ReduceSinkOperator.class, dedupCtx.trustScript()); + if (pRS != null && merge(cRS, pRS, dedupCtx.minReducer())) { + CorrelationUtilities.removeReduceSinkForGroupBy(cRS, cGBY, dedupCtx.getPctx(), dedupCtx); + return true; + } + return false; + } + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java @@ -416,6 +416,22 @@ copyReducerConf(mapJoinTask, childTask); } + public static boolean cannotConvert(String bigTableAlias, + Map aliasToSize, long aliasTotalKnownInputSize, + long ThresholdOfSmallTblSizeSum) { + boolean ret = false; + Long aliasKnownSize = aliasToSize.get(bigTableAlias); + if (aliasKnownSize != null && aliasKnownSize.longValue() > 0) { + long smallTblTotalKnownSize = aliasTotalKnownInputSize + - aliasKnownSize.longValue(); + if (smallTblTotalKnownSize > ThresholdOfSmallTblSizeSum) { + //this table is not good to be a big table. + ret = true; + } + } + return ret; + } + @Override public Task processCurrentTask(MapRedTask currTask, ConditionalTask conditionalTask, Context context) @@ -564,14 +580,9 @@ MapRedTask newTask = newTaskAlias.getFirst(); bigTableAlias = newTaskAlias.getSecond(); - Long aliasKnownSize = aliasToSize.get(bigTableAlias); - if (aliasKnownSize != null && aliasKnownSize.longValue() > 0) { - long smallTblTotalKnownSize = aliasTotalKnownInputSize - - aliasKnownSize.longValue(); - if (smallTblTotalKnownSize > ThresholdOfSmallTblSizeSum) { - // this table is not good to be a big table. - continue; - } + if (cannotConvert(bigTableAlias, aliasToSize, + aliasTotalKnownInputSize, ThresholdOfSmallTblSizeSum)) { + continue; } // add into conditional task Index: ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -3499,7 +3499,8 @@ } List> distinctColIndices = getDistinctColIndicesForReduceSink(parseInfo, dest, - reduceKeys, reduceSinkInputRowResolver, reduceSinkOutputRowResolver, outputKeyColumnNames); + reduceKeys, reduceSinkInputRowResolver, reduceSinkOutputRowResolver, outputKeyColumnNames, + colExprMap); ArrayList reduceValues = new ArrayList(); HashMap aggregationTrees = parseInfo @@ -3572,7 +3573,8 @@ private List> getDistinctColIndicesForReduceSink(QBParseInfo parseInfo, String dest, List reduceKeys, RowResolver reduceSinkInputRowResolver, - RowResolver reduceSinkOutputRowResolver, List outputKeyColumnNames) + RowResolver reduceSinkOutputRowResolver, List outputKeyColumnNames, + Map colExprMap) throws SemanticException { List> distinctColIndices = new ArrayList>(); @@ -3611,6 +3613,7 @@ ColumnInfo colInfo = new ColumnInfo(field, expr.getTypeInfo(), null, false); reduceSinkOutputRowResolver.putExpression(parameter, colInfo); numExprs++; + colExprMap.put(colInfo.getInternalName(), expr); } distinctColIndices.add(distinctIndices); } @@ -3672,7 +3675,8 @@ colExprMap); List> distinctColIndices = getDistinctColIndicesForReduceSink(parseInfo, dest, - reduceKeys, reduceSinkInputRowResolver, reduceSinkOutputRowResolver, outputKeyColumnNames); + reduceKeys, reduceSinkInputRowResolver, reduceSinkOutputRowResolver, outputKeyColumnNames, + colExprMap); ArrayList reduceValues = new ArrayList(); @@ -6952,6 +6956,7 @@ reduceValues.size() - 1).getTypeInfo(), "", false); reduceSinkOutputRowResolver.putExpression(grpbyExpr, colInfo); outputColumnNames.add(getColumnInternalName(reduceValues.size() - 1)); + colExprMap.put(colInfo.getInternalName(), grpByExprNode); } } Index: ql/src/java/org/apache/hadoop/hive/ql/plan/DemuxDesc.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/DemuxDesc.java @@ -0,0 +1,96 @@ +/** + * 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.util.List; +import java.util.Map; + + +/** + * Demux operator descriptor implementation. + * + */ +@Explain(displayName = "Demux Operator") +public class DemuxDesc extends AbstractOperatorDesc { + + private static final long serialVersionUID = 1L; + + private Map newTagToOldTag; + private Map newTagToChildIndex; + private List keysSerializeInfos; + private List valuesSerializeInfos; + private Map childIndexToOriginalNumParents; + + public DemuxDesc() { + } + + public DemuxDesc( + Map newTagToOldTag, + Map newTagToChildIndex, + Map childIndexToOriginalNumParents, + List keysSerializeInfos, + List valuesSerializeInfos){ + this.newTagToOldTag = newTagToOldTag; + this.newTagToChildIndex = newTagToChildIndex; + this.childIndexToOriginalNumParents = childIndexToOriginalNumParents; + this.keysSerializeInfos = keysSerializeInfos; + this.valuesSerializeInfos = valuesSerializeInfos; + } + + public List getKeysSerializeInfos() { + return keysSerializeInfos; + } + + public void setKeysSerializeInfos(List keysSerializeInfos) { + this.keysSerializeInfos = keysSerializeInfos; + } + + public List getValuesSerializeInfos() { + return valuesSerializeInfos; + } + + public void setValuesSerializeInfos(List valuesSerializeInfos) { + this.valuesSerializeInfos = valuesSerializeInfos; + } + + public Map getNewTagToOldTag() { + return newTagToOldTag; + } + + public void setNewTagToOldTag(Map newTagToOldTag) { + this.newTagToOldTag = newTagToOldTag; + } + + public Map getNewTagToChildIndex() { + return newTagToChildIndex; + } + + public void setNewTagToChildIndex(Map newTagToChildIndex) { + this.newTagToChildIndex = newTagToChildIndex; + } + + public Map getChildIndexToOriginalNumParents() { + return childIndexToOriginalNumParents; + } + + public void setChildIndexToOriginalNumParents( + Map childIndexToOriginalNumParents) { + this.childIndexToOriginalNumParents = childIndexToOriginalNumParents; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/plan/MuxDesc.java =================================================================== --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MuxDesc.java @@ -0,0 +1,129 @@ +/** + * 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.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; + + +/** + * Mux operator descriptor implementation.. + * + */ +@Explain(displayName = "Mux Operator") +public class MuxDesc extends AbstractOperatorDesc { + private static final long serialVersionUID = 1L; + + private List> parentToKeyCols; + private List> parentToValueCols; + private List> parentToOutputKeyColumnNames; + private List> parentToOutputValueColumnNames; + private List parentToTag; + private Map newParentIndexToOldParentIndex; + + public MuxDesc() { + } + + // A MuxDesc is only generated from a corresponding ReduceSinkDesc. + public MuxDesc(List> ops){ + int numParents = ops.size(); + parentToKeyCols = new ArrayList>(numParents); + parentToOutputKeyColumnNames = new ArrayList>(numParents); + parentToValueCols = new ArrayList>(numParents); + parentToOutputValueColumnNames = new ArrayList>(numParents); + parentToTag = new ArrayList(numParents); + + for (Operator op: ops) { + if (op != null && op instanceof ReduceSinkOperator) { + ReduceSinkOperator rsop = (ReduceSinkOperator)op; + List keyCols = rsop.getConf().getKeyCols(); + List valueCols = rsop.getConf().getValueCols(); + List outputKeyColumnNames = rsop.getConf().getOutputKeyColumnNames(); + List outputValueColumnNames = rsop.getConf().getOutputValueColumnNames(); + int tag = rsop.getConf().getTag(); + parentToKeyCols.add(keyCols); + parentToValueCols.add(valueCols); + parentToOutputKeyColumnNames.add(outputKeyColumnNames); + parentToOutputValueColumnNames.add(outputValueColumnNames); + parentToTag.add(tag); + } else { + parentToKeyCols.add(null); + parentToValueCols.add(null); + parentToOutputKeyColumnNames.add(null); + parentToOutputValueColumnNames.add(null); + parentToTag.add(null); + } + } + } + + public List> getParentToKeyCols() { + return parentToKeyCols; + } + + public void setParentToKeyCols(List> parentToKeyCols) { + this.parentToKeyCols = parentToKeyCols; + } + + public List> getParentToValueCols() { + return parentToValueCols; + } + + public void setParentToValueCols(List> parentToValueCols) { + this.parentToValueCols = parentToValueCols; + } + + public List> getParentToOutputKeyColumnNames() { + return parentToOutputKeyColumnNames; + } + + public void setParentToOutputKeyColumnNames( + List> parentToOutputKeyColumnNames) { + this.parentToOutputKeyColumnNames = parentToOutputKeyColumnNames; + } + + public List> getParentToOutputValueColumnNames() { + return parentToOutputValueColumnNames; + } + + public void setParentToOutputValueColumnNames( + List> parentToOutputValueColumnNames) { + this.parentToOutputValueColumnNames = parentToOutputValueColumnNames; + } + + public List getParentToTag() { + return parentToTag; + } + + public void setParentToTag(List parentToTag) { + this.parentToTag = parentToTag; + } + + public Map getNewParentIndexToOldParentIndex() { + return newParentIndexToOldParentIndex; + } + + public void setNewParentIndexToOldParentIndex( + Map newParentIndexToOldParentIndex) { + this.newParentIndexToOldParentIndex = newParentIndexToOldParentIndex; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java @@ -71,13 +71,13 @@ public ReduceSinkDesc() { } - public ReduceSinkDesc(java.util.ArrayList keyCols, + public ReduceSinkDesc(ArrayList keyCols, int numDistributionKeys, - java.util.ArrayList valueCols, - java.util.ArrayList outputKeyColumnNames, + ArrayList valueCols, + ArrayList outputKeyColumnNames, List> distinctColumnIndices, - java.util.ArrayList outputValueColumnNames, int tag, - java.util.ArrayList partitionCols, int numReducers, + ArrayList outputValueColumnNames, int tag, + ArrayList partitionCols, int numReducers, final TableDesc keySerializeInfo, final TableDesc valueSerializeInfo) { this.keyCols = keyCols; this.numDistributionKeys = numDistributionKeys; Index: ql/src/java/org/apache/hadoop/hive/ql/plan/UnionDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/UnionDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/UnionDesc.java @@ -26,12 +26,16 @@ @Explain(displayName = "Union") public class UnionDesc extends AbstractOperatorDesc { private static final long serialVersionUID = 1L; - private transient int numInputs; + // If this UnionOperator is inside the reduce side of an MR job generated + // by Correlation Optimizer, which means all inputs of this UnionOperator are + // from DemuxOperator. If so, we should not touch this UnionOperator in genMapRedTasks. + private transient boolean allInputsInSameReducer; @SuppressWarnings("nls") public UnionDesc() { numInputs = 2; + allInputsInSameReducer = false; } /** @@ -48,4 +52,12 @@ public void setNumInputs(int numInputs) { this.numInputs = numInputs; } + + public boolean isAllInputsInSameReducer() { + return allInputsInSameReducer; + } + + public void setAllInputsInSameReducer(boolean allInputsInSameReducer) { + this.allInputsInSameReducer = allInputsInSameReducer; + } } Index: ql/src/test/queries/clientpositive/correlationoptimizer1.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer1.q @@ -0,0 +1,281 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- This query has a GroupByOperator folling JoinOperator and they share the same keys. +-- When Correlation Optimizer is turned off, three MR jobs will be generated. +-- When Correlation Optimizer is turned on, two MR jobs will be generated +-- and JoinOperator (on the column of key) and GroupByOperator (also on the column +-- of key) will be executed in the first MR job. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=true; +set hive.optimize.correlation=true; +-- Enable hive.auto.convert.join. +-- Correlation Optimizer will detect that the join will be converted to a Map-join, +-- so it will not try to optimize this query. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=true; +set hive.optimize.mapjoin.mapreduce=true; +set hive.optimize.correlation=true; +-- Enable hive.auto.convert.join. +-- Correlation Optimizer will detect that the join will be converted to a Map-join, +-- so it will not try to optimize this query. +-- We should generate 1 MR job for subquery tmp. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- If the key of a GroupByOperator is the left table's key in +-- a Left Semi Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- If the key of a GroupByOperator is the left table's key in +-- a Left Outer Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + + +set hive.optimize.correlation=false; +-- If the key of a GroupByOperator is the right table's key in +-- a Left Outer Join, we cannot use a single MR to execute these two +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=false; +-- If the key of a GroupByOperator is the right table's key in +-- a Right Outer Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp; + + +set hive.optimize.correlation=false; +-- If the key of a GroupByOperator is the left table's key in +-- a Right Outer Join, we cannot use a single MR to execute these two +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=false; +-- This query has a Full Outer Join followed by a GroupByOperator and +-- they share the same key. Because those keys with a null value are not grouped +-- in the output of the Full Outer Join, we cannot use a single MR to execute +-- these two operators. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- Currently, we only handle exactly same keys, this query will not be optimized +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp; + +set hive.optimize.correlation=false; +-- Currently, we only handle exactly same keys, this query will not be optimized +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp; Index: ql/src/test/queries/clientpositive/correlationoptimizer10.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer10.q @@ -0,0 +1,130 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +-- This case is used to test LEFT SEMI JOIN since Hive will +-- introduce a GroupByOperator before the ReduceSinkOperator of +-- the right table (yy in queries below) +-- of LEFT SEMI JOIN. +EXPLAIN +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt; + +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt; + +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +-- This case is used to test LEFT SEMI JOIN since Hive will +-- introduce a GroupByOperator before the ReduceSinkOperator of +-- the right table (yy in queries below) +-- of LEFT SEMI JOIN. +EXPLAIN +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- This test is used to test if we can use shared scan for +-- xx, yy:x, and yy:y. +EXPLAIN +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; + +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value; Index: ql/src/test/queries/clientpositive/correlationoptimizer11.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer11.q @@ -0,0 +1,73 @@ +set hive.auto.convert.join=false; +-- Tests in this file are used to make sure Correlation Optimizer +-- can correctly handle tables with partitions + +CREATE TABLE part_table(key string, value string) PARTITIONED BY (partitionId int); +INSERT OVERWRITE TABLE part_table PARTITION (partitionId=1) + SELECT key, value FROM src ORDER BY key, value LIMIT 100; +INSERT OVERWRITE TABLE part_table PARTITION (partitionId=2) + SELECT key, value FROM src1 ORDER BY key, value; + +set hive.optimize.correlation=false; +-- In this case, we should not do shared scan on part_table +-- because left and right tables of JOIN use different partitions +-- of part_table. With Correlation Optimizer we will generate +-- 1 MR job. +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key; + +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key; + +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key; + +set hive.optimize.correlation=false; +-- In this case, we should do shared scan on part_table +-- because left and right tables of JOIN use the same partition +-- of part_table. With Correlation Optimizer we will generate +-- 1 MR job. +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key; + +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key; + +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key; Index: ql/src/test/queries/clientpositive/correlationoptimizer12.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer12.q @@ -0,0 +1,9 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=true; +-- Currently, correlation optimizer does not support PTF operator +EXPLAIN SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(x.value) OVER (PARTITION BY x.key) AS cnt FROM src x) xx +JOIN +(SELECT y.key as key, count(y.value) OVER (PARTITION BY y.key) AS cnt FROM src1 y) yy +ON (xx.key=yy.key); Index: ql/src/test/queries/clientpositive/correlationoptimizer13.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer13.q @@ -0,0 +1,18 @@ +CREATE TABLE tmp(c1 INT, c2 INT, c3 STRING, c4 STRING); +INSERT OVERWRITE TABLE tmp +SELECT x.key, y.key, x.value, y.value FROM src x JOIN src y ON (x.key = y.key); + +set hive.optimize.correlation=true; +-- The query in this file have operators with same set of keys +-- but having different sorting orders. +-- Correlation optimizer currently do not optimize this case. +-- This case will be optimized latter (need a follow-up jira). + +EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c3, x.c1) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c3, x1.c1) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; + Index: ql/src/test/queries/clientpositive/correlationoptimizer14.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer14.q @@ -0,0 +1,149 @@ +set hive.optimize.reducededuplication=true; +set hive.optimize.reducededuplication.min.reducer=1; +set hive.optimize.correlation=true; +-- This file is used to show plans of queries involving cluster by, distribute by, +-- order by, and sort by. +-- Right now, Correlation optimizer check the most restrictive condition +-- when determining if a ReduceSinkOperator is not necessary. +-- This condition is that two ReduceSinkOperators should have same sorting columns, +-- same partitioning columns, same sorting orders and no conflict on the numbers of reducers. + +-- Distribute by will not be optimized because distribute by does not introduce +-- sorting columns. +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key) yy +ON (xx.key=yy.key); + +-- Sort by will not be optimized because sort by does not introduce partitioning columns +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y SORT BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=false; +-- Distribute by and sort by on the same key(s) should be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=true; +-- Because for join we use ascending order, if sort by uses descending order, +-- this query will not be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key DESC) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key DESC) yy +ON (xx.key=yy.key); + +-- Even if hive.optimize.reducededuplication.min.reducer=1, order by will not be optimized +-- because order by does not introduce partitioning columns +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x ORDER BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y ORDER BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=false; +-- Cluster by will be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=false; +-- If hive.optimize.reducededuplication.min.reducer=1, +-- group by and then order by should be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key); + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key); + +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key); Index: ql/src/test/queries/clientpositive/correlationoptimizer2.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer2.q @@ -0,0 +1,188 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- In this query, subquery a and b both have a GroupByOperator and the a and b will be +-- joined. The key of JoinOperator is the same with both keys of GroupByOperators in subquery +-- a and b. When Correlation Optimizer is turned off, we have four MR jobs. +-- When Correlation Optimizer is turned on, 2 MR jobs will be generated. +-- The first job will evaluate subquery tmp (including subquery a, b, and the JoinOperator on a +-- and b). +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=false; +-- Left Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=false; +-- Right Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=false; +-- Full Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=false; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp; + +set hive.optimize.correlation=true; +-- After FULL OUTER JOIN, keys with null values are not grouped, right now, +-- we have to generate 2 MR jobs for tmp, 1 MR job for a join b and another for the +-- GroupByOperator on key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, we need 4 MR jobs. +-- When Correlation Optimizer is turned on, the subquery of tmp will be evaluated in +-- a single MR job (including the subquery a, the subquery b, and a join b). So, we +-- will have 2 MR jobs. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp; + +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp; Index: ql/src/test/queries/clientpositive/correlationoptimizer3.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer3.q @@ -0,0 +1,98 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 5 MR jobs will be generated. +-- When Correlation Optimizer is turned on, the subquery tmp will be evalauted +-- in a single MR job (including the subquery b, the subquery d, and b join d). +-- At the reduce side of the MR job evaluating tmp, two operation paths +-- (for subquery b and d) have different depths. The path starting from subquery b +-- is JOIN->GBY->JOIN, which has a depth of 3. While, the path starting from subquery d +-- is JOIN->JOIN. We should be able to handle this case. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +set hive.optimize.mapjoin.mapreduce=true; +-- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +set hive.optimize.mapjoin.mapreduce=true; +-- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp; Index: ql/src/test/queries/clientpositive/correlationoptimizer4.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer4.q @@ -0,0 +1,159 @@ +CREATE TABLE T1(key INT, val STRING); +LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +CREATE TABLE T2(key INT, val STRING); +LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +CREATE TABLE T3(key INT, val STRING); +LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T3; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, this query will be evaluated +-- by 3 MR jobs. +-- When Correlation Optimizer is turned on, this query will be evaluated by +-- 2 MR jobs. The subquery tmp will be evaluated in a single MR job. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +set hive.optimize.mapjoin.mapreduce=true; +-- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- This case should be optimized, since the key of GroupByOperator is from the leftmost table +-- of a chain of LEFT OUTER JOINs. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp; + +set hive.optimize.correlation=true; +-- This query will not be optimized by correlation optimizer because +-- GroupByOperator uses y.key (a right table of a left outer join) +-- as the key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=false; +-- This case should be optimized, since the key of GroupByOperator is from the rightmost table +-- of a chain of RIGHT OUTER JOINs. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp; + +set hive.optimize.correlation=true; +-- This query will not be optimized by correlation optimizer because +-- GroupByOperator uses y.key (a left table of a right outer join) +-- as the key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=false; +-- This case should not be optimized because afer the FULL OUTER JOIN, rows with null keys +-- are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; + +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp; Index: ql/src/test/queries/clientpositive/correlationoptimizer5.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer5.q @@ -0,0 +1,80 @@ +CREATE TABLE T1(key INT, val STRING); +LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; +CREATE TABLE T2(key INT, val STRING); +LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T2; +CREATE TABLE T3(key INT, val STRING); +LOAD DATA LOCAL INPATH '../data/files/kv3.txt' INTO TABLE T3; +CREATE TABLE T4(key INT, val STRING); +LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE T4; + +CREATE TABLE dest_co1(key INT, val STRING); +CREATE TABLE dest_co2(key INT, val STRING); +CREATE TABLE dest_co3(key INT, val STRING); + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 3 MR jobs are needed. +-- When Correlation Optimizer is turned on, only a single MR job is needed. +EXPLAIN +INSERT OVERWRITE TABLE dest_co1 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +INSERT OVERWRITE TABLE dest_co1 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +set hive.optimize.correlation=true; +EXPLAIN +INSERT OVERWRITE TABLE dest_co2 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +INSERT OVERWRITE TABLE dest_co2 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +set hive.optimize.mapjoin.mapreduce=true; +set hive.auto.convert.join.noconditionaltask.size=10000000000; +-- Enable hive.auto.convert.join. +EXPLAIN +INSERT OVERWRITE TABLE dest_co3 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +INSERT OVERWRITE TABLE dest_co3 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key; + +-- dest_co1, dest_co2 and dest_co3 should be same +-- SELECT * FROM dest_co1 x ORDER BY x.key, x.val; +-- SELECT * FROM dest_co2 x ORDER BY x.key, x.val; +SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co1; +SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co2; +SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co3; Index: ql/src/test/queries/clientpositive/correlationoptimizer6.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer6.q @@ -0,0 +1,324 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx, subquery yy, and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +set hive.optimize.mapjoin.mapreduce=true; +-- Enable hive.auto.convert.join. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 3 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt; + +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery tmp and tmp join z. +EXPLAIN +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value; + +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value; + +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value; + +set hive.optimize.correlation=false; +-- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 4 MR jobs are needed. +-- 2 MR jobs are used to evaluate yy, 1 MR is used to evaluate xx and xx join yy. +-- The last MR is used for ordering. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +set hive.optimize.correlation=true; +set hive.auto.convert.join=true; +set hive.optimize.mapjoin.mapreduce=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt; + Index: ql/src/test/queries/clientpositive/correlationoptimizer7.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer7.q @@ -0,0 +1,82 @@ +set hive.auto.convert.join=true; + +set hive.optimize.correlation=false; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000000000; +set hive.optimize.mapjoin.mapreduce=true; + +set hive.optimize.correlation=false; +-- Without correlation optimizer, we will have 3 MR jobs. +-- The first one is a MapJoin and Aggregation (in the Reduce Phase). +-- The second one is another MapJoin. The third one is for ordering. +-- With the correlation optimizer, right now, we still have +-- 3 MR jobs. The first one is a MapJoin and the map-side aggregation (a map-only job). +-- The second one have the reduce-side aggregation and the second join. +-- The third one is for ordering. +-- Although we have turned on hive.optimize.mapjoin.mapreduce, that optimizer +-- can not handle the case that the MR job (the one which a map-only job will be merged in) +-- has multiple inputs. We should improve that optimizer. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value; + Index: ql/src/test/queries/clientpositive/correlationoptimizer8.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer8.q @@ -0,0 +1,110 @@ +set hive.auto.convert.join=false; +set hive.optimize.correlation=false; +-- When the Correlation Optimizer is turned off, this query will be evaluated by +-- 4 MR jobs. +-- When the Correlation Optimizer is turned on, because both inputs of the +-- UnionOperator are correlated, we can use 2 MR jobs to evaluate this query. +-- The first MR job will evaluate subquery subq1 and subq1 join x. The second +-- MR is for ordering. +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +set hive.optimize.correlation=false; +-- When the Correlation Optimizer is turned off, this query will be evaluated by +-- 4 MR jobs. +-- When the Correlation Optimizer is turned on, because both inputs of the +-- UnionOperator are correlated, we can use 2 MR jobs to evaluate this query. +-- The first MR job will evaluate subquery subq1 and subq1 join x. The second +-- MR is for ordering. +EXPLAIN +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value; + +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value; + +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value; + +set hive.optimize.correlation=true; +-- When the Correlation Optimizer is turned on, because a input of UnionOperator is +-- not correlated, we cannot handle this case right now. So, this query will not be +-- optimized. +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key, x1.value +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt; + +set hive.optimize.correlation=true; +-- When the Correlation Optimizer is turned on, because a input of UnionOperator is +-- not correlated, we cannot handle this case right now. So, this query will not be +-- optimized. +EXPLAIN +SELECT subq1.key, subq1.value, x.key, x.value +FROM +( SELECT cast(x.key as INT) as key, count(1) as value from src x where x.key < 20 group by x.key + UNION ALL + SELECT count(1) as key, cast(x1.key as INT) as value from src x1 where x1.key > 100 group by x1.key +) subq1 +FULL OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.value, x.key, x.value; Index: ql/src/test/queries/clientpositive/correlationoptimizer9.q =================================================================== --- /dev/null +++ ql/src/test/queries/clientpositive/correlationoptimizer9.q @@ -0,0 +1,71 @@ +CREATE TABLE tmp(c1 INT, c2 INT, c3 STRING, c4 STRING); + +set hive.auto.convert.join=false; + +INSERT OVERWRITE TABLE tmp +SELECT x.key, y.key, x.value, y.value FROM src x JOIN src y ON (x.key = y.key); + +set hive.optimize.correlation=false; +EXPLAIN +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt; + +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt; + +set hive.optimize.correlation=true; +-- The merged table scan should be able to load both c1 and c2 +EXPLAIN +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt; + +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt; + +set hive.optimize.correlation=false; +EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; + +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; + +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt; Index: ql/src/test/results/clientpositive/correlationoptimizer1.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer1.q.out @@ -0,0 +1,3710 @@ +PREHOOK: query: -- This query has a GroupByOperator folling JoinOperator and they share the same keys. +-- When Correlation Optimizer is turned off, three MR jobs will be generated. +-- When Correlation Optimizer is turned on, two MR jobs will be generated +-- and JoinOperator (on the column of key) and GroupByOperator (also on the column +-- of key) will be executed in the first MR job. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- This query has a GroupByOperator folling JoinOperator and they share the same keys. +-- When Correlation Optimizer is turned off, three MR jobs will be generated. +-- When Correlation Optimizer is turned on, two MR jobs will be generated +-- and JoinOperator (on the column of key) and GroupByOperator (also on the column +-- of key) will be executed in the first MR job. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 37 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 37 +PREHOOK: query: -- Enable hive.auto.convert.join. +-- Correlation Optimizer will detect that the join will be converted to a Map-join, +-- so it will not try to optimize this query. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Enable hive.auto.convert.join. +-- Correlation Optimizer will detect that the join will be converted to a Map-join, +-- so it will not try to optimize this query. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-6 is a root stage + Stage-5 depends on stages: Stage-6 + Stage-2 depends on stages: Stage-5 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-6 + Map Reduce Local Work + Alias -> Map Local Tables: + tmp:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:x + TableScan + alias: x + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + tmp:y + TableScan + alias: y + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 37 +PREHOOK: query: -- Enable hive.auto.convert.join. +-- Correlation Optimizer will detect that the join will be converted to a Map-join, +-- so it will not try to optimize this query. +-- We should generate 1 MR job for subquery tmp. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Enable hive.auto.convert.join. +-- Correlation Optimizer will detect that the join will be converted to a Map-join, +-- so it will not try to optimize this query. +-- We should generate 1 MR job for subquery tmp. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-6 is a root stage + Stage-5 depends on stages: Stage-6 + Stage-3 depends on stages: Stage-5 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-6 + Map Reduce Local Work + Alias -> Map Local Tables: + tmp:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:x + TableScan + alias: x + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + tmp:y + TableScan + alias: y + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 37 +PREHOOK: query: -- If the key of a GroupByOperator is the left table's key in +-- a Left Semi Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- If the key of a GroupByOperator is the left table's key in +-- a Left Semi Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTSEMIJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 15 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTSEMIJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT SEMI JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 15 +PREHOOK: query: -- If the key of a GroupByOperator is the left table's key in +-- a Left Outer Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- If the key of a GroupByOperator is the left table's key in +-- a Left Outer Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 47 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 47 +PREHOOK: query: -- If the key of a GroupByOperator is the right table's key in +-- a Left Outer Join, we cannot use a single MR to execute these two +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- If the key of a GroupByOperator is the right table's key in +-- a Left Outer Join, we cannot use a single MR to execute these two +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: string + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 47 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 + 1 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: string + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 47 +PREHOOK: query: -- If the key of a GroupByOperator is the right table's key in +-- a Right Outer Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- If the key of a GroupByOperator is the right table's key in +-- a Right Outer Join, these two operators will be executed in +-- the same MR job when Correlation Optimizer is enabled. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 + 1 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: string + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +12744278 500 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 + 1 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: string + outputColumnNames: _col4 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +12744278 500 +PREHOOK: query: -- If the key of a GroupByOperator is the left table's key in +-- a Right Outer Join, we cannot use a single MR to execute these two +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- If the key of a GroupByOperator is the left table's key in +-- a Right Outer Join, we cannot use a single MR to execute these two +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 500 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x RIGHT OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 500 +PREHOOK: query: -- This query has a Full Outer Join followed by a GroupByOperator and +-- they share the same key. Because those keys with a null value are not grouped +-- in the output of the Full Outer Join, we cannot use a single MR to execute +-- these two operators. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- This query has a Full Outer Join followed by a GroupByOperator and +-- they share the same key. Because those keys with a null value are not grouped +-- in the output of the Full Outer Join, we cannot use a single MR to execute +-- these two operators. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 510 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x FULL OUTER JOIN src y ON (x.key = y.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 510 +PREHOOK: query: -- Currently, we only handle exactly same keys, this query will not be optimized +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Currently, we only handle exactly same keys, this query will not be optimized +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL x) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + expr: _col1 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 661329102 37 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL x) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + expr: _col1 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.value)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, x.value AS value, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key) + GROUP BY x.key, x.value) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 661329102 37 +PREHOOK: query: -- Currently, we only handle exactly same keys, this query will not be optimized +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Currently, we only handle exactly same keys, this query will not be optimized +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (AND (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)) (= (. (TOK_TABLE_OR_COL x) value) (. (TOK_TABLE_OR_COL y) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + expr: value + type: string + sort order: ++ + Map-reduce partition columns: + expr: key + type: string + expr: value + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + expr: value + type: string + sort order: ++ + Map-reduce partition columns: + expr: key + type: string + expr: value + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +503358 29 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (AND (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)) (= (. (TOK_TABLE_OR_COL x) value) (. (TOK_TABLE_OR_COL y) value))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + expr: value + type: string + sort order: ++ + Map-reduce partition columns: + expr: key + type: string + expr: value + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + expr: value + type: string + sort order: ++ + Map-reduce partition columns: + expr: key + type: string + expr: value + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src1 x JOIN src y ON (x.key = y.key AND x.value = y.value) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +503358 29 Index: ql/src/test/results/clientpositive/correlationoptimizer10.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer10.q.out @@ -0,0 +1,1374 @@ +PREHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +-- This case is used to test LEFT SEMI JOIN since Hive will +-- introduce a GroupByOperator before the ReduceSinkOperator of +-- the right table (yy in queries below) +-- of LEFT SEMI JOIN. +EXPLAIN +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +-- This case is used to test LEFT SEMI JOIN since Hive will +-- introduce a GroupByOperator before the ReduceSinkOperator of +-- the right table (yy in queries below) +-- of LEFT SEMI JOIN. +EXPLAIN +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTSEMIJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt))))) + +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-4 depends on stages: Stage-3 + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + xx:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + yy + TableScan + alias: yy + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + sort order: ++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTSEMIJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + xx:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + yy + TableScan + alias: yy + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 2 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + sort order: ++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +LEFT SEMI JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 +PREHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +-- This case is used to test LEFT SEMI JOIN since Hive will +-- introduce a GroupByOperator before the ReduceSinkOperator of +-- the right table (yy in queries below) +-- of LEFT SEMI JOIN. +EXPLAIN +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +-- This case is used to test LEFT SEMI JOIN since Hive will +-- introduce a GroupByOperator before the ReduceSinkOperator of +-- the right table (yy in queries below) +-- of LEFT SEMI JOIN. +EXPLAIN +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTSEMIJOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key)) (TOK_WHERE (AND (< (. (TOK_TABLE_OR_COL x) key) 200) (> (. (TOK_TABLE_OR_COL y) key) 20))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Filter Operator + predicate: + expr: ((key < 200.0) and (key > 20.0)) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Filter Operator + predicate: + expr: ((key > 20.0) and (key < 200.0)) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 +146 val_146 +150 val_150 +66 val_66 +98 val_98 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTSEMIJOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key)) (TOK_WHERE (AND (< (. (TOK_TABLE_OR_COL x) key) 200) (> (. (TOK_TABLE_OR_COL y) key) 20))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + yy:x + TableScan + alias: x + Filter Operator + predicate: + expr: ((key < 200.0) and (key > 20.0)) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Filter Operator + predicate: + expr: ((key > 20.0) and (key < 200.0)) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + Reduce Operator Tree: + Demux Operator + Mux Operator + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value +FROM +src1 xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND + y.key > 20) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 +146 val_146 +150 val_150 +66 val_66 +98 val_98 +PREHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- This test is used to test if we can use shared scan for +-- xx, yy:x, and yy:y. +EXPLAIN +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- This test is used to test if we can use shared scan for +-- xx, yy:x, and yy:y. +EXPLAIN +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTSEMIJOIN (TOK_TABREF (TOK_TABNAME src) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key)) (TOK_WHERE (AND (< (. (TOK_TABLE_OR_COL x) key) 200) (> (. (TOK_TABLE_OR_COL x) key) 180))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Filter Operator + predicate: + expr: ((key < 200.0) and (key > 180.0)) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Filter Operator + predicate: + expr: ((key < 200.0) and (key > 180.0)) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTSEMIJOIN (TOK_TABREF (TOK_TABNAME src) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key)) (TOK_WHERE (AND (< (. (TOK_TABLE_OR_COL x) key) 200) (> (. (TOK_TABLE_OR_COL x) key) 180))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + yy:x + TableScan + alias: x + Filter Operator + predicate: + expr: ((key < 200.0) and (key > 180.0)) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Filter Operator + predicate: + expr: ((key < 200.0) and (key > 180.0)) + type: boolean + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + Reduce Operator Tree: + Demux Operator + Mux Operator + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Join Operator + condition map: + Left Semi Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value +FROM +src xx +LEFT SEMI JOIN +(SELECT x.key as key + FROM src x JOIN src y ON (x.key = y.key) + WHERE x.key < 200 AND x.key > 180) yy +ON xx.key=yy.key ORDER BY xx.key, xx.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +#### A masked pattern was here #### +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 Index: ql/src/test/results/clientpositive/correlationoptimizer11.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer11.q.out @@ -0,0 +1,647 @@ +PREHOOK: query: -- Tests in this file are used to make sure Correlation Optimizer +-- can correctly handle tables with partitions + +CREATE TABLE part_table(key string, value string) PARTITIONED BY (partitionId int) +PREHOOK: type: CREATETABLE +POSTHOOK: query: -- Tests in this file are used to make sure Correlation Optimizer +-- can correctly handle tables with partitions + +CREATE TABLE part_table(key string, value string) PARTITIONED BY (partitionId int) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@part_table +PREHOOK: query: INSERT OVERWRITE TABLE part_table PARTITION (partitionId=1) + SELECT key, value FROM src ORDER BY key, value LIMIT 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@part_table@partitionid=1 +POSTHOOK: query: INSERT OVERWRITE TABLE part_table PARTITION (partitionId=1) + SELECT key, value FROM src ORDER BY key, value LIMIT 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@part_table@partitionid=1 +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: INSERT OVERWRITE TABLE part_table PARTITION (partitionId=2) + SELECT key, value FROM src1 ORDER BY key, value +PREHOOK: type: QUERY +PREHOOK: Input: default@src1 +PREHOOK: Output: default@part_table@partitionid=2 +POSTHOOK: query: INSERT OVERWRITE TABLE part_table PARTITION (partitionId=2) + SELECT key, value FROM src1 ORDER BY key, value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src1 +POSTHOOK: Output: default@part_table@partitionid=2 +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- In this case, we should not do shared scan on part_table +-- because left and right tables of JOIN use different partitions +-- of part_table. With Correlation Optimizer we will generate +-- 1 MR job. +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key +PREHOOK: type: QUERY +POSTHOOK: query: -- In this case, we should not do shared scan on part_table +-- because left and right tables of JOIN use different partitions +-- of part_table. With Correlation Optimizer we will generate +-- 1 MR job. +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME part_table) x) (TOK_TABREF (TOK_TABNAME part_table) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (AND (= (. (TOK_TABLE_OR_COL x) partitionId) 1) (= (. (TOK_TABLE_OR_COL y) partitionId) 2))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key +PREHOOK: type: QUERY +PREHOOK: Input: default@part_table +PREHOOK: Input: default@part_table@partitionid=1 +PREHOOK: Input: default@part_table@partitionid=2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_table +POSTHOOK: Input: default@part_table@partitionid=1 +POSTHOOK: Input: default@part_table@partitionid=2 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +128 3 +146 2 +150 1 +PREHOOK: query: EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME part_table) x) (TOK_TABREF (TOK_TABNAME part_table) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (AND (= (. (TOK_TABLE_OR_COL x) partitionId) 1) (= (. (TOK_TABLE_OR_COL y) partitionId) 2))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) + +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: + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key +PREHOOK: type: QUERY +PREHOOK: Input: default@part_table +PREHOOK: Input: default@part_table@partitionid=1 +PREHOOK: Input: default@part_table@partitionid=2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 1 AND + y.partitionId = 2 +GROUP BY x.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_table +POSTHOOK: Input: default@part_table@partitionid=1 +POSTHOOK: Input: default@part_table@partitionid=2 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +128 3 +146 2 +150 1 +PREHOOK: query: -- In this case, we should do shared scan on part_table +-- because left and right tables of JOIN use the same partition +-- of part_table. With Correlation Optimizer we will generate +-- 1 MR job. +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key +PREHOOK: type: QUERY +POSTHOOK: query: -- In this case, we should do shared scan on part_table +-- because left and right tables of JOIN use the same partition +-- of part_table. With Correlation Optimizer we will generate +-- 1 MR job. +EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME part_table) x) (TOK_TABREF (TOK_TABNAME part_table) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (AND (= (. (TOK_TABLE_OR_COL x) partitionId) 2) (= (. (TOK_TABLE_OR_COL y) partitionId) 2))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key +PREHOOK: type: QUERY +PREHOOK: Input: default@part_table +PREHOOK: Input: default@part_table@partitionid=2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_table +POSTHOOK: Input: default@part_table@partitionid=2 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] + 100 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 +PREHOOK: query: EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME part_table) x) (TOK_TABREF (TOK_TABNAME part_table) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (AND (= (. (TOK_TABLE_OR_COL x) partitionId) 2) (= (. (TOK_TABLE_OR_COL y) partitionId) 2))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) + +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: + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key +PREHOOK: type: QUERY +PREHOOK: Input: default@part_table +PREHOOK: Input: default@part_table@partitionid=2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT x.key AS key, count(1) AS cnt +FROM part_table x JOIN part_table y ON (x.key = y.key) +WHERE x.partitionId = 2 AND + y.partitionId = 2 +GROUP BY x.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part_table +POSTHOOK: Input: default@part_table@partitionid=2 +#### A masked pattern was here #### +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: part_table PARTITION(partitionid=2).value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] + 100 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 +66 1 +98 1 Index: ql/src/test/results/clientpositive/correlationoptimizer12.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer12.q.out @@ -0,0 +1,169 @@ +PREHOOK: query: -- Currently, correlation optimizer does not support PTF operator +EXPLAIN SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(x.value) OVER (PARTITION BY x.key) AS cnt FROM src x) xx +JOIN +(SELECT y.key as key, count(y.value) OVER (PARTITION BY y.key) AS cnt FROM src1 y) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- Currently, correlation optimizer does not support PTF operator +EXPLAIN SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(x.value) OVER (PARTITION BY x.key) AS cnt FROM src x) xx +JOIN +(SELECT y.key as key, count(y.value) OVER (PARTITION BY y.key) AS cnt FROM src1 y) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value) (TOK_WINDOWSPEC (TOK_PARTITIONINGSPEC (TOK_DISTRIBUTEBY (. (TOK_TABLE_OR_COL x) key))))) cnt (TOK_WINDOWSPEC (TOK_PARTITIONINGSPEC (TOK_DISTRIBUTEBY (. (TOK_TABLE_OR_COL x) key)))))))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value) (TOK_WINDOWSPEC (TOK_PARTITIONINGSPEC (TOK_DISTRIBUTEBY (. (TOK_TABLE_OR_COL y) key))))) cnt (TOK_WINDOWSPEC (TOK_PARTITIONINGSPEC (TOK_DISTRIBUTEBY (. (TOK_TABLE_OR_COL y) key)))))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + expr: key + type: string + sort order: ++ + Map-reduce partition columns: + expr: key + type: string + tag: -1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Extract + PTF Operator + Select Operator + expressions: + expr: _col0 + type: string + expr: _wcol0 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + 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-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + expr: key + type: string + sort order: ++ + Map-reduce partition columns: + expr: key + type: string + tag: -1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Extract + PTF Operator + Select Operator + expressions: + expr: _col0 + type: string + expr: _wcol0 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + Index: ql/src/test/results/clientpositive/correlationoptimizer13.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer13.q.out @@ -0,0 +1,327 @@ +PREHOOK: query: CREATE TABLE tmp(c1 INT, c2 INT, c3 STRING, c4 STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE tmp(c1 INT, c2 INT, c3 STRING, c4 STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@tmp +PREHOOK: query: INSERT OVERWRITE TABLE tmp +SELECT x.key, y.key, x.value, y.value FROM src x JOIN src y ON (x.key = y.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tmp +POSTHOOK: query: INSERT OVERWRITE TABLE tmp +SELECT x.key, y.key, x.value, y.value FROM src x JOIN src y ON (x.key = y.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tmp +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: -- The query in this file have operators with same set of keys +-- but having different sorting orders. +-- Correlation optimizer currently do not optimize this case. +-- This case will be optimized latter (need a follow-up jira). + +EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c3, x.c1) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c3, x1.c1) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- The query in this file have operators with same set of keys +-- but having different sorting orders. +-- Correlation optimizer currently do not optimize this case. +-- This case will be optimized latter (need a follow-up jira). + +EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c3, x.c1) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c3, x1.c1) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) c1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) c3) key2) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) c1) 120)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) c3) (. (TOK_TABLE_OR_COL x) c1)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) c1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) c3) key2) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) c2) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) c3) (. (TOK_TABLE_OR_COL x1) c1)))) yy) (AND (= (. (TOK_TABLE_OR_COL xx) key1) (. (TOK_TABLE_OR_COL yy) key1)) (== (. (TOK_TABLE_OR_COL xx) key2) (. (TOK_TABLE_OR_COL yy) key2))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key2)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key2)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (c2 > 100) + type: boolean + Select Operator + expressions: + expr: c3 + type: string + expr: c1 + type: int + outputColumnNames: c3, c1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c3 + type: string + expr: c1 + type: int + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: int + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: int + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col1 + type: int + expr: _col0 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: int + expr: _col1 + type: string + tag: 1 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: int + expr: _col1 + type: string + tag: 0 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col3 + type: int + expr: _col4 + type: string + expr: _col2 + type: bigint + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: bigint + expr: _col5 + type: bigint + sort order: ++++++ + tag: -1 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: bigint + expr: _col5 + type: bigint + Reduce Operator Tree: + Extract + 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-4 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Filter Operator + predicate: + expr: (c1 < 120) + type: boolean + Select Operator + expressions: + expr: c3 + type: string + expr: c1 + type: int + outputColumnNames: c3, c1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c3 + type: string + expr: c1 + type: int + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: int + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: int + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col1 + type: int + expr: _col0 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + Index: ql/src/test/results/clientpositive/correlationoptimizer14.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer14.q.out @@ -0,0 +1,1952 @@ +PREHOOK: query: -- This file is used to show plans of queries involving cluster by, distribute by, +-- order by, and sort by. +-- Right now, Correlation optimizer check the most restrictive condition +-- when determining if a ReduceSinkOperator is not necessary. +-- This condition is that two ReduceSinkOperators should have same sorting columns, +-- same partitioning columns, same sorting orders and no conflict on the numbers of reducers. + +-- Distribute by will not be optimized because distribute by does not introduce +-- sorting columns. +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- This file is used to show plans of queries involving cluster by, distribute by, +-- order by, and sort by. +-- Right now, Correlation optimizer check the most restrictive condition +-- when determining if a ReduceSinkOperator is not necessary. +-- This condition is that two ReduceSinkOperators should have same sorting columns, +-- same partitioning columns, same sorting orders and no conflict on the numbers of reducers. + +-- Distribute by will not be optimized because distribute by does not introduce +-- sorting columns. +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + sort order: + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + sort order: + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: -- Sort by will not be optimized because sort by does not introduce partitioning columns +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y SORT BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- Sort by will not be optimized because sort by does not introduce partitioning columns +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y SORT BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: -- Distribute by and sort by on the same key(s) should be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- Distribute by and sort by on the same key(s) should be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 val_128 128 +128 val_128 128 +128 val_128 128 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +213 val_213 213 val_213 +213 val_213 213 val_213 +224 val_224 224 +224 val_224 224 +238 val_238 238 val_238 +238 val_238 238 val_238 +255 val_255 255 val_255 +255 val_255 255 val_255 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +278 val_278 278 val_278 +278 val_278 278 val_278 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +369 val_369 369 +369 val_369 369 +369 val_369 369 +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 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +66 val_66 66 val_66 +98 val_98 98 val_98 +98 val_98 98 val_98 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +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: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Demux Operator + Extract + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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 + Extract + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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 xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 val_128 128 +128 val_128 128 +128 val_128 128 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +213 val_213 213 val_213 +213 val_213 213 val_213 +224 val_224 224 +224 val_224 224 +238 val_238 238 val_238 +238 val_238 238 val_238 +255 val_255 255 val_255 +255 val_255 255 val_255 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +278 val_278 278 val_278 +278 val_278 278 val_278 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +369 val_369 369 +369 val_369 369 +369 val_369 369 +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 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +66 val_66 66 val_66 +98 val_98 98 val_98 +98 val_98 98 val_98 +PREHOOK: query: -- Because for join we use ascending order, if sort by uses descending order, +-- this query will not be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key DESC) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key DESC) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- Because for join we use ascending order, if sort by uses descending order, +-- this query will not be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x DISTRIBUTE BY key SORT BY key DESC) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y DISTRIBUTE BY key SORT BY key DESC) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)) (TOK_SORTBY (TOK_TABSORTCOLNAMEDESC (TOK_TABLE_OR_COL key))))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)) (TOK_SORTBY (TOK_TABSORTCOLNAMEDESC (TOK_TABLE_OR_COL key))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: - + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: - + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: -- Even if hive.optimize.reducededuplication.min.reducer=1, order by will not be optimized +-- because order by does not introduce partitioning columns +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x ORDER BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y ORDER BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- Even if hive.optimize.reducededuplication.min.reducer=1, order by will not be optimized +-- because order by does not introduce partitioning columns +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x ORDER BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y ORDER BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: -- Cluster by will be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- Cluster by will be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_CLUSTERBY (TOK_TABLE_OR_COL key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value)) (TOK_CLUSTERBY (TOK_TABLE_OR_COL key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 val_128 128 +128 val_128 128 +128 val_128 128 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +213 val_213 213 val_213 +213 val_213 213 val_213 +224 val_224 224 +224 val_224 224 +238 val_238 238 val_238 +238 val_238 238 val_238 +255 val_255 255 val_255 +255 val_255 255 val_255 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +278 val_278 278 val_278 +278 val_278 278 val_278 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +369 val_369 369 +369 val_369 369 +369 val_369 369 +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 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +66 val_66 66 val_66 +98 val_98 98 val_98 +98 val_98 98 val_98 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_CLUSTERBY (TOK_TABLE_OR_COL key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) value) value)) (TOK_CLUSTERBY (TOK_TABLE_OR_COL key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +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: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Demux Operator + Extract + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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 + Extract + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + 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 xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x Cluster BY key) xx +JOIN +(SELECT y.key as key, y.value as value FROM src1 y Cluster BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 val_128 128 +128 val_128 128 +128 val_128 128 +146 val_146 146 val_146 +146 val_146 146 val_146 +150 val_150 150 val_150 +213 val_213 213 val_213 +213 val_213 213 val_213 +224 val_224 224 +224 val_224 224 +238 val_238 238 val_238 +238 val_238 238 val_238 +255 val_255 255 val_255 +255 val_255 255 val_255 +273 val_273 273 val_273 +273 val_273 273 val_273 +273 val_273 273 val_273 +278 val_278 278 val_278 +278 val_278 278 val_278 +311 val_311 311 val_311 +311 val_311 311 val_311 +311 val_311 311 val_311 +369 val_369 369 +369 val_369 369 +369 val_369 369 +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 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +406 val_406 406 val_406 +66 val_66 66 val_66 +98 val_98 98 val_98 +98 val_98 98 val_98 +PREHOOK: query: -- If hive.optimize.reducededuplication.min.reducer=1, +-- group by and then order by should be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: -- If hive.optimize.reducededuplication.min.reducer=1, +-- group by and then order by should be optimized +EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_CLUSTERBY (TOK_TABLE_OR_COL key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) value)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-3 + Stage-3 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + expr: _col3 + type: bigint + 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-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Extract + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 val_128 128 1 +128 val_128 128 1 +128 val_128 128 1 +146 val_146 146 1 +146 val_146 146 1 +150 val_150 150 1 +213 val_213 213 1 +213 val_213 213 1 +224 val_224 224 1 +224 val_224 224 1 +238 val_238 238 1 +238 val_238 238 1 +255 val_255 255 1 +255 val_255 255 1 +273 val_273 273 1 +273 val_273 273 1 +273 val_273 273 1 +278 val_278 278 1 +278 val_278 278 1 +311 val_311 311 1 +311 val_311 311 1 +311 val_311 311 1 +369 val_369 369 1 +369 val_369 369 1 +369 val_369 369 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +406 val_406 406 1 +406 val_406 406 1 +406 val_406 406 1 +406 val_406 406 1 +66 val_66 66 1 +98 val_98 98 1 +98 val_98 98 1 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value)) (TOK_CLUSTERBY (TOK_TABLE_OR_COL key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) value)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))))) + +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: + xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Extract + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + expr: _col3 + type: bigint + 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 + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + expr: _col3 + type: bigint + 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 xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.value, yy.key, yy.value +FROM +(SELECT x.key as key, x.value as value FROM src x CLUSTER BY key) xx +JOIN +(SELECT y.key as key, count(*) as value FROM src1 y GROUP BY y.key ORDER BY key) yy +ON (xx.key=yy.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 val_128 128 1 +128 val_128 128 1 +128 val_128 128 1 +146 val_146 146 1 +146 val_146 146 1 +150 val_150 150 1 +213 val_213 213 1 +213 val_213 213 1 +224 val_224 224 1 +224 val_224 224 1 +238 val_238 238 1 +238 val_238 238 1 +255 val_255 255 1 +255 val_255 255 1 +273 val_273 273 1 +273 val_273 273 1 +273 val_273 273 1 +278 val_278 278 1 +278 val_278 278 1 +311 val_311 311 1 +311 val_311 311 1 +311 val_311 311 1 +369 val_369 369 1 +369 val_369 369 1 +369 val_369 369 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +401 val_401 401 1 +406 val_406 406 1 +406 val_406 406 1 +406 val_406 406 1 +406 val_406 406 1 +66 val_66 66 1 +98 val_98 98 1 +98 val_98 98 1 Index: ql/src/test/results/clientpositive/correlationoptimizer2.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer2.q.out @@ -0,0 +1,3311 @@ +PREHOOK: query: -- In this query, subquery a and b both have a GroupByOperator and the a and b will be +-- joined. The key of JoinOperator is the same with both keys of GroupByOperators in subquery +-- a and b. When Correlation Optimizer is turned off, we have four MR jobs. +-- When Correlation Optimizer is turned on, 2 MR jobs will be generated. +-- The first job will evaluate subquery tmp (including subquery a, b, and the JoinOperator on a +-- and b). +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- In this query, subquery a and b both have a GroupByOperator and the a and b will be +-- joined. The key of JoinOperator is the same with both keys of GroupByOperators in subquery +-- a and b. When Correlation Optimizer is turned off, we have four MR jobs. +-- When Correlation Optimizer is turned on, 2 MR jobs will be generated. +-- The first job will evaluate subquery tmp (including subquery a, b, and the JoinOperator on a +-- and b). +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) cnt) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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-4 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 37 652447 15 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) cnt) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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 SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 37 652447 15 +PREHOOK: query: -- Left Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Left Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) cnt) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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-4 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +12744278 500 652447 15 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) cnt) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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 SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + LEFT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +12744278 500 652447 15 +PREHOOK: query: -- Right Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Right Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) cnt) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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-4 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 37 652447 25 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) cnt) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Right Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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 SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + RIGHT OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +652447 37 652447 25 +PREHOOK: query: -- Full Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Full Outer Join should be handled. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) cnt) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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-4 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +12744278 500 652447 25 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) cnt) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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 SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.cnt AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +12744278 500 652447 25 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL a) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-5 + Stage-3 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-3 + Stage-5 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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-5 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +12744278 310 +PREHOOK: query: -- After FULL OUTER JOIN, keys with null values are not grouped, right now, +-- we have to generate 2 MR jobs for tmp, 1 MR job for a join b and another for the +-- GroupByOperator on key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- After FULL OUTER JOIN, keys with null values are not grouped, right now, +-- we have to generate 2 MR jobs for tmp, 1 MR job for a join b and another for the +-- GroupByOperator on key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL x) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL y) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL a) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + tmp:b:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT a.key AS key, count(1) AS cnt + FROM (SELECT x.key as key, count(x.value) AS cnt FROM src x group by x.key) a + FULL OUTER JOIN (SELECT y.key as key, count(y.value) AS cnt FROM src1 y group by y.key) b + ON (a.key = b.key) + GROUP BY a.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +12744278 310 +PREHOOK: query: -- When Correlation Optimizer is turned off, we need 4 MR jobs. +-- When Correlation Optimizer is turned on, the subquery of tmp will be evaluated in +-- a single MR job (including the subquery a, the subquery b, and a join b). So, we +-- will have 2 MR jobs. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, we need 4 MR jobs. +-- When Correlation Optimizer is turned on, the subquery of tmp will be evaluated in +-- a single MR job (including the subquery a, the subquery b, and a join b). So, we +-- will have 2 MR jobs. +EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) val)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) z)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL z) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL z) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) val) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:z + TableScan + alias: z + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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-4 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + tmp:a:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +1711763 3531902962 1711763 37 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) val)))) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) z)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) key) key) (TOK_SELEXPR (TOK_FUNCTION count (. (TOK_TABLE_OR_COL z) value)) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL z) key)))) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) val) cnt1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt2)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt1)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL key2)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (TOK_TABLE_OR_COL cnt2))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:a:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + tmp:a:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + tmp:b:z + TableScan + alias: z + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(value) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 2 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + expr: sum(hash(_col3)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + expr: sum(VALUE._col3) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: bigint + 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 SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key1)), SUM(HASH(cnt1)), SUM(HASH(key2)), SUM(HASH(cnt2)) +FROM (SELECT a.key AS key1, a.val AS cnt1, b.key AS key2, b.cnt AS cnt2 + FROM (SELECT x.key AS key, x.value AS val FROM src1 x JOIN src y ON (x.key = y.key)) a + JOIN (SELECT z.key AS key, count(z.value) AS cnt FROM src1 z group by z.key) b + ON (a.key = b.key)) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +1711763 3531902962 1711763 37 Index: ql/src/test/results/clientpositive/correlationoptimizer3.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer3.q.out @@ -0,0 +1,1782 @@ +PREHOOK: query: -- When Correlation Optimizer is turned off, 5 MR jobs will be generated. +-- When Correlation Optimizer is turned on, the subquery tmp will be evalauted +-- in a single MR job (including the subquery b, the subquery d, and b join d). +-- At the reduce side of the MR job evaluating tmp, two operation paths +-- (for subquery b and d) have different depths. The path starting from subquery b +-- is JOIN->GBY->JOIN, which has a depth of 3. While, the path starting from subquery d +-- is JOIN->JOIN. We should be able to handle this case. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 5 MR jobs will be generated. +-- When Correlation Optimizer is turned on, the subquery tmp will be evalauted +-- in a single MR job (including the subquery b, the subquery d, and b join d). +-- At the reduce side of the MR job evaluating tmp, two operation paths +-- (for subquery b and d) have different depths. The path starting from subquery b +-- is JOIN->GBY->JOIN, which has a depth of 3. While, the path starting from subquery d +-- is JOIN->JOIN. We should be able to handle this case. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) value) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2, Stage-6 + Stage-4 depends on stages: Stage-3 + Stage-6 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:b:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + 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-6 + Map Reduce + Alias -> Map Operator Tree: + tmp:d:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + tmp:d:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +1711763 107 3531902962 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) value) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:b:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + tmp:d:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + value expressions: + expr: key + type: string + expr: value + type: string + tmp:d:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 3 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +1711763 107 3531902962 +PREHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) cnt) cnt) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) value) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value))))))) + +STAGE DEPENDENCIES: + Stage-9 is a root stage + Stage-7 depends on stages: Stage-9 + Stage-2 depends on stages: Stage-7, Stage-8 + Stage-3 depends on stages: Stage-2 + Stage-10 is a root stage + Stage-8 depends on stages: Stage-10 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-9 + Map Reduce Local Work + Alias -> Map Local Tables: + tmp:b:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:b:x + TableScan + alias: x + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + + Stage: Stage-7 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:y + TableScan + alias: y + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: string + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + 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-10 + Map Reduce Local Work + Alias -> Map Local Tables: + tmp:d:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:d:x + TableScan + alias: x + HashTable Sink Operator + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + tmp:d:y + TableScan + alias: y + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT b.key AS key, b.cnt AS cnt, d.value AS value + FROM (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) b + JOIN (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +1711763 107 3531902962 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) cnt) cnt) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-6 + Stage-3 depends on stages: Stage-2 + Stage-5 is a root stage + Stage-6 depends on stages: Stage-5 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + tmp:b:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: bigint + expr: _col1 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + 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-5 + Map Reduce + Alias -> Map Operator Tree: + tmp:d:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + tmp:d:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +1711763 107 3531902962 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) cnt) cnt) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + tmp:b:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + tmp:d:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + value expressions: + expr: key + type: string + tmp:d:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 3 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: bigint + expr: _col1 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: bigint + expr: _col1 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +1711763 107 3531902962 +PREHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) cnt) cnt) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value) value)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) value))))))) + +STAGE DEPENDENCIES: + Stage-9 is a root stage + Stage-7 depends on stages: Stage-9 + Stage-2 depends on stages: Stage-7, Stage-8 + Stage-3 depends on stages: Stage-2 + Stage-10 is a root stage + Stage-8 depends on stages: Stage-10 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-9 + Map Reduce Local Work + Alias -> Map Local Tables: + tmp:b:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:b:x + TableScan + alias: x + HashTable Sink Operator + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + + Stage: Stage-7 + Map Reduce + Alias -> Map Operator Tree: + tmp:b:y + TableScan + alias: y + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: string +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: bigint + expr: _col1 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: bigint + expr: _col1 + type: string + outputColumnNames: _col0, _col1, _col2 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + expr: sum(hash(_col2)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + expr: sum(VALUE._col2) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + 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-10 + Map Reduce Local Work + Alias -> Map Local Tables: + tmp:d:x + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:d:x + TableScan + alias: x + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + tmp:d:y + TableScan + alias: y + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)), SUM(HASH(tmp.value)) +FROM (SELECT d.key AS key, d.cnt AS cnt, b.value as value + FROM (SELECT x.key, x.value FROM src1 x JOIN src y ON (x.key = y.key)) b + JOIN (SELECT x.key, count(1) AS cnt FROM src1 x JOIN src y ON (x.key = y.key) group by x.key) d + ON b.key = d.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +1711763 107 3531902962 Index: ql/src/test/results/clientpositive/correlationoptimizer4.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer4.q.out @@ -0,0 +1,2180 @@ +PREHOOK: query: CREATE TABLE T1(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T1(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t1 +PREHOOK: query: CREATE TABLE T2(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T2(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T2 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2 +PREHOOK: type: LOAD +PREHOOK: Output: default@t2 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t2 +PREHOOK: query: CREATE TABLE T3(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T3(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T3 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T3 +PREHOOK: type: LOAD +PREHOOK: Output: default@t3 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T3 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t3 +PREHOOK: query: -- When Correlation Optimizer is turned off, this query will be evaluated +-- by 3 MR jobs. +-- When Correlation Optimizer is turned on, this query will be evaluated by +-- 2 MR jobs. The subquery tmp will be evaluated in a single MR job. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, this query will be evaluated +-- by 3 MR jobs. +-- When Correlation Optimizer is turned on, this query will be evaluated by +-- 2 MR jobs. The subquery tmp will be evaluated in a single MR job. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + value expressions: + expr: key + type: int + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 + 1 {VALUE._col0} + 2 + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: int + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: int + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +13 10 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + value expressions: + expr: key + type: int + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 + 1 {VALUE._col0} + 2 + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: int + outputColumnNames: _col4 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: int + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +13 10 +PREHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-7 is a root stage + Stage-6 depends on stages: Stage-7 + Stage-3 depends on stages: Stage-6 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + tmp:y + Fetch Operator + limit: -1 + tmp:z + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + tmp:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 + 1 {key} + 2 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + Position of Big Table: 0 + tmp:z + TableScan + alias: z + HashTable Sink Operator + condition expressions: + 0 + 1 {key} + 2 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 + 1 {key} + 2 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + 2 [Column[key]] + outputColumnNames: _col4 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col4 + type: int + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: int + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x JOIN T1 y ON (x.key = y.key) JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +13 10 +PREHOOK: query: -- This case should be optimized, since the key of GroupByOperator is from the leftmost table +-- of a chain of LEFT OUTER JOINs. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- This case should be optimized, since the key of GroupByOperator is from the leftmost table +-- of a chain of LEFT OUTER JOINs. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + value expressions: + expr: key + type: int + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 {VALUE._col0} + 1 + 2 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: int + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: int + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +22 12 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + value expressions: + expr: key + type: int + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 {VALUE._col0} + 1 + 2 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: int + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: int + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT x.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY x.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +22 12 +PREHOOK: query: -- This query will not be optimized by correlation optimizer because +-- GroupByOperator uses y.key (a right table of a left outer join) +-- as the key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- This query will not be optimized by correlation optimizer because +-- GroupByOperator uses y.key (a right table of a left outer join) +-- as the key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + value expressions: + expr: key + type: int + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + Left Outer Join1 to 2 + condition expressions: + 0 + 1 {VALUE._col0} + 2 + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: int + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: int + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x LEFT OUTER JOIN T1 y ON (x.key = y.key) LEFT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +13 12 +PREHOOK: query: -- This case should be optimized, since the key of GroupByOperator is from the rightmost table +-- of a chain of RIGHT OUTER JOINs. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- This case should be optimized, since the key of GroupByOperator is from the rightmost table +-- of a chain of RIGHT OUTER JOINs. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL z) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + value expressions: + expr: key + type: int + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 + 1 + 2 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col8 + Select Operator + expressions: + expr: _col8 + type: int + outputColumnNames: _col8 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col8 + type: int + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +21 12 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL z) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + value expressions: + expr: key + type: int + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Right Outer Join0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 + 1 + 2 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col8 + Select Operator + expressions: + expr: _col8 + type: int + outputColumnNames: _col8 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col8 + type: int + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT z.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY z.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +21 12 +PREHOOK: query: -- This query will not be optimized by correlation optimizer because +-- GroupByOperator uses y.key (a left table of a right outer join) +-- as the key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- This query will not be optimized by correlation optimizer because +-- GroupByOperator uses y.key (a left table of a right outer join) +-- as the key. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + value expressions: + expr: key + type: int + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + Reduce Operator Tree: + Join Operator + condition map: + Right Outer Join0 to 1 + Right Outer Join1 to 2 + condition expressions: + 0 + 1 {VALUE._col0} + 2 + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: int + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: int + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x RIGHT OUTER JOIN T1 y ON (x.key = y.key) RIGHT OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +21 12 +PREHOOK: query: -- This case should not be optimized because afer the FULL OUTER JOIN, rows with null keys +-- are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: -- This case should not be optimized because afer the FULL OUTER JOIN, rows with null keys +-- are not grouped. +EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_FULLOUTERJOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + value expressions: + expr: key + type: int + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + Outer Join 1 to 2 + condition expressions: + 0 + 1 {VALUE._col0} + 2 + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: int + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: int + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +21 14 +PREHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_FULLOUTERJOIN (TOK_TABREF (TOK_TABNAME T2) x) (TOK_TABREF (TOK_TABNAME T1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key))) (TOK_TABREF (TOK_TABNAME T3) z) (= (. (TOK_TABLE_OR_COL y) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) tmp)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) key)))) (TOK_SELEXPR (TOK_FUNCTION SUM (TOK_FUNCTION HASH (. (TOK_TABLE_OR_COL tmp) cnt))))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + tmp:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + value expressions: + expr: key + type: int + tmp:z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + Outer Join 1 to 2 + condition expressions: + 0 + 1 {VALUE._col0} + 2 + handleSkewJoin: false + outputColumnNames: _col4 + Select Operator + expressions: + expr: _col4 + type: int + outputColumnNames: _col4 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col4 + type: int + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: sum(hash(_col0)) + expr: sum(hash(_col1)) + bucketGroup: false + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + sort order: + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + 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 SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) +FROM (SELECT y.key AS key, count(1) AS cnt + FROM T2 x FULL OUTER JOIN T1 y ON (x.key = y.key) FULL OUTER JOIN T3 z ON (y.key = z.key) + GROUP BY y.key) tmp +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +#### A masked pattern was here #### +21 14 Index: ql/src/test/results/clientpositive/correlationoptimizer5.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer5.q.out @@ -0,0 +1,913 @@ +PREHOOK: query: CREATE TABLE T1(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T1(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T1 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1 +PREHOOK: type: LOAD +PREHOOK: Output: default@t1 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t1 +PREHOOK: query: CREATE TABLE T2(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T2(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T2 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T2 +PREHOOK: type: LOAD +PREHOOK: Output: default@t2 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T2 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t2 +PREHOOK: query: CREATE TABLE T3(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T3(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T3 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/kv3.txt' INTO TABLE T3 +PREHOOK: type: LOAD +PREHOOK: Output: default@t3 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/kv3.txt' INTO TABLE T3 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t3 +PREHOOK: query: CREATE TABLE T4(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE T4(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@T4 +PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE T4 +PREHOOK: type: LOAD +PREHOOK: Output: default@t4 +POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE T4 +POSTHOOK: type: LOAD +POSTHOOK: Output: default@t4 +PREHOOK: query: CREATE TABLE dest_co1(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE dest_co1(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@dest_co1 +PREHOOK: query: CREATE TABLE dest_co2(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE dest_co2(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@dest_co2 +PREHOOK: query: CREATE TABLE dest_co3(key INT, val STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE dest_co3(key INT, val STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@dest_co3 +PREHOOK: query: -- When Correlation Optimizer is turned off, 3 MR jobs are needed. +-- When Correlation Optimizer is turned on, only a single MR job is needed. +EXPLAIN +INSERT OVERWRITE TABLE dest_co1 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 3 MR jobs are needed. +-- When Correlation Optimizer is turned on, only a single MR job is needed. +EXPLAIN +INSERT OVERWRITE TABLE dest_co1 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) x) (TOK_TABREF (TOK_TABNAME T2) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) val))))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T3) m) (TOK_TABREF (TOK_TABNAME T4) n) (= (. (TOK_TABLE_OR_COL m) key) (. (TOK_TABLE_OR_COL n) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL m) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL n) val))))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_co1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) val))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-0 depends on stages: Stage-2 + Stage-3 depends on stages: Stage-0 + Stage-4 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + d:m + TableScan + alias: m + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + value expressions: + expr: key + type: int + d:n + TableScan + alias: n + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + value expressions: + expr: val + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: 1 + value expressions: + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: 0 + value expressions: + expr: _col0 + type: int + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 1 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co1 + + Stage: Stage-3 + Stats-Aggr Operator + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: + b:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + value expressions: + expr: key + type: int + b:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: int + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + +PREHOOK: query: INSERT OVERWRITE TABLE dest_co1 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: default@dest_co1 +POSTHOOK: query: INSERT OVERWRITE TABLE dest_co1 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: default@dest_co1 +POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: EXPLAIN +INSERT OVERWRITE TABLE dest_co2 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +INSERT OVERWRITE TABLE dest_co2 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) x) (TOK_TABREF (TOK_TABNAME T2) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) val))))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T3) m) (TOK_TABREF (TOK_TABNAME T4) n) (= (. (TOK_TABLE_OR_COL m) key) (. (TOK_TABLE_OR_COL n) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL m) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL n) val))))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_co2))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) val))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-2 depends on stages: Stage-0 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + b:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 0 + value expressions: + expr: key + type: int + b:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 1 + d:m + TableScan + alias: m + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 2 + value expressions: + expr: key + type: int + d:n + TableScan + alias: n + Reduce Output Operator + key expressions: + expr: key + type: int + sort order: + + Map-reduce partition columns: + expr: key + type: int + tag: 3 + value expressions: + expr: val + type: string + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: int + outputColumnNames: _col0 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 1 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co2 + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 1 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co2 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co2 + + Stage: Stage-2 + Stats-Aggr Operator + + +PREHOOK: query: INSERT OVERWRITE TABLE dest_co2 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: default@dest_co2 +POSTHOOK: query: INSERT OVERWRITE TABLE dest_co2 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: default@dest_co2 +POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co2.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co2.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +INSERT OVERWRITE TABLE dest_co3 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +PREHOOK: type: QUERY +POSTHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +INSERT OVERWRITE TABLE dest_co3 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +POSTHOOK: type: QUERY +POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co2.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co2.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) x) (TOK_TABREF (TOK_TABNAME T2) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) val))))) b) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T3) m) (TOK_TABREF (TOK_TABNAME T4) n) (= (. (TOK_TABLE_OR_COL m) key) (. (TOK_TABLE_OR_COL n) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL m) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL n) val))))) d) (= (. (TOK_TABLE_OR_COL b) key) (. (TOK_TABLE_OR_COL d) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_co3))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL d) val))))) + +STAGE DEPENDENCIES: + Stage-14 is a root stage + Stage-10 depends on stages: Stage-14 + Stage-9 depends on stages: Stage-10, Stage-11 , consists of Stage-12, Stage-13, Stage-2 + Stage-12 has a backup stage: Stage-2 + Stage-7 depends on stages: Stage-12 + Stage-0 depends on stages: Stage-2, Stage-7, Stage-8 + Stage-3 depends on stages: Stage-0 + Stage-13 has a backup stage: Stage-2 + Stage-8 depends on stages: Stage-13 + Stage-2 + Stage-15 is a root stage + Stage-11 depends on stages: Stage-15 + +STAGE PLANS: + Stage: Stage-14 + Map Reduce Local Work + Alias -> Map Local Tables: + d:m + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + d:m + TableScan + alias: m + HashTable Sink Operator + condition expressions: + 0 {key} + 1 {val} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 1 + + Stage: Stage-10 + Map Reduce + Alias -> Map Operator Tree: + d:n + TableScan + alias: n + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 {val} + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col5 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col5 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-9 + Conditional Operator + + Stage: Stage-12 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + HashTable Sink Operator + condition expressions: + 0 {_col0} + 1 {_col1} + handleSkewJoin: false + keys: + 0 [Column[_col0]] + 1 [Column[_col0]] + Position of Big Table: 0 + + Stage: Stage-7 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME1 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {_col0} + 1 {_col1} + handleSkewJoin: false + keys: + 0 [Column[_col0]] + 1 [Column[_col0]] + outputColumnNames: _col0, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 1 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co3 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co3 + + Stage: Stage-3 + Stats-Aggr Operator + + Stage: Stage-13 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME1 + HashTable Sink Operator + condition expressions: + 0 {_col0} + 1 {_col1} + handleSkewJoin: false + keys: + 0 [Column[_col0]] + 1 [Column[_col0]] + Position of Big Table: 1 + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {_col0} + 1 {_col1} + handleSkewJoin: false + keys: + 0 [Column[_col0]] + 1 [Column[_col0]] + outputColumnNames: _col0, _col3 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 1 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co3 + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: 1 + value expressions: + expr: _col1 + type: string + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: 0 + value expressions: + expr: _col0 + type: int + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col3 + type: string + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 1 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.dest_co3 + + Stage: Stage-15 + Map Reduce Local Work + Alias -> Map Local Tables: + b:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-11 + Map Reduce + Alias -> Map Operator Tree: + b:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: int + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + +PREHOOK: query: INSERT OVERWRITE TABLE dest_co3 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +PREHOOK: type: QUERY +PREHOOK: Input: default@t1 +PREHOOK: Input: default@t2 +PREHOOK: Input: default@t3 +PREHOOK: Input: default@t4 +PREHOOK: Output: default@dest_co3 +POSTHOOK: query: INSERT OVERWRITE TABLE dest_co3 +SELECT b.key, d.val +FROM +(SELECT x.key, x.val FROM T1 x JOIN T2 y ON (x.key = y.key)) b +JOIN +(SELECT m.key, n.val FROM T3 m JOIN T4 n ON (m.key = n.key)) d +ON b.key = d.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@t1 +POSTHOOK: Input: default@t2 +POSTHOOK: Input: default@t3 +POSTHOOK: Input: default@t4 +POSTHOOK: Output: default@dest_co3 +POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co2.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co2.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co3.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co3.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +PREHOOK: query: -- dest_co1, dest_co2 and dest_co3 should be same +-- SELECT * FROM dest_co1 x ORDER BY x.key, x.val; +-- SELECT * FROM dest_co2 x ORDER BY x.key, x.val; +SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co1 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest_co1 +#### A masked pattern was here #### +POSTHOOK: query: -- dest_co1, dest_co2 and dest_co3 should be same +-- SELECT * FROM dest_co1 x ORDER BY x.key, x.val; +-- SELECT * FROM dest_co2 x ORDER BY x.key, x.val; +SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest_co1 +#### A masked pattern was here #### +POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co2.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co2.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co3.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co3.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +8409 7619696771 +PREHOOK: query: SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co2 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest_co2 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest_co2 +#### A masked pattern was here #### +POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co2.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co2.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co3.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co3.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +8409 7619696771 +PREHOOK: query: SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co3 +PREHOOK: type: QUERY +PREHOOK: Input: default@dest_co3 +#### A masked pattern was here #### +POSTHOOK: query: SELECT SUM(HASH(key)), SUM(HASH(val)) FROM dest_co3 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dest_co3 +#### A masked pattern was here #### +POSTHOOK: Lineage: dest_co1.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co1.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co2.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co2.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +POSTHOOK: Lineage: dest_co3.key EXPRESSION [(t1)x.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: dest_co3.val SIMPLE [(t4)n.FieldSchema(name:val, type:string, comment:null), ] +8409 7619696771 Index: ql/src/test/results/clientpositive/correlationoptimizer6.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer6.q.out @@ -0,0 +1,5482 @@ +PREHOOK: query: -- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx, subquery yy, and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx, subquery yy, and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2, Stage-7 + Stage-4 depends on stages: Stage-3 + Stage-6 is a root stage + Stage-7 depends on stages: Stage-6 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + Reduce Operator Tree: + Extract + 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-6 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + xx:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-7 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 128 9 +146 1 146 4 +150 1 150 1 +213 1 213 4 +224 1 224 4 +238 1 238 4 +255 1 255 4 +273 1 273 9 +278 1 278 4 +311 1 311 9 +369 1 369 9 +401 1 401 25 +406 1 406 16 +66 1 66 1 +98 1 98 4 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + xx:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 3 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 128 9 +146 1 146 4 +150 1 150 1 +213 1 213 4 +224 1 224 4 +238 1 238 4 +255 1 255 4 +273 1 273 9 +278 1 278 4 +311 1 311 9 +369 1 369 9 +401 1 401 25 +406 1 406 16 +66 1 66 1 +98 1 98 4 +PREHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- Enable hive.auto.convert.join. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-9 is a root stage + Stage-7 depends on stages: Stage-9 + Stage-2 depends on stages: Stage-7, Stage-8 + Stage-3 depends on stages: Stage-2 + Stage-10 is a root stage + Stage-8 depends on stages: Stage-10 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-9 + Map Reduce Local Work + Alias -> Map Local Tables: + yy:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + yy:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-7 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: bigint + Reduce Operator Tree: + Extract + 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-10 + Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 128 9 +146 1 146 4 +150 1 150 1 +213 1 213 4 +224 1 224 4 +238 1 238 4 +255 1 255 4 +273 1 273 9 +278 1 278 4 +311 1 311 9 +369 1 369 9 +401 1 401 25 +406 1 406 16 +66 1 66 1 +98 1 98 4 +PREHOOK: query: -- When Correlation Optimizer is turned off, 3 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 3 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 128 3 +146 146 2 +150 150 1 +213 213 2 +224 224 2 +238 238 2 +255 255 2 +273 273 3 +278 278 2 +311 311 3 +369 369 3 +401 401 5 +406 406 4 +66 66 1 +98 98 2 +PREHOOK: query: EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x GROUP BY x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 128 3 +146 146 2 +150 150 1 +213 213 2 +224 224 2 +238 238 2 +255 255 2 +273 273 3 +278 278 2 +311 311 3 +369 369 3 +401 401 5 +406 406 4 +66 66 1 +98 98 2 +PREHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-3 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 128 9 +146 146 4 +150 150 1 +213 213 4 +224 224 4 +238 238 4 +255 255 4 +273 273 9 +278 278 4 +311 311 9 +369 369 9 +401 401 25 +406 406 16 +66 66 1 +98 98 4 +PREHOOK: query: EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + Reduce Operator Tree: + Demux Operator + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 128 9 +146 146 4 +150 150 1 +213 213 4 +224 224 4 +238 238 4 +255 255 4 +273 273 9 +278 278 4 +311 311 9 +369 369 9 +401 401 25 +406 406 16 +66 66 1 +98 98 4 +PREHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key))))) + +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-4 depends on stages: Stage-3 + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + xx:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + yy + TableScan + alias: yy + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 128 +128 1 128 +128 1 128 +146 1 146 +146 1 146 +150 1 150 +213 1 213 +213 1 213 +224 1 224 +224 1 224 +238 1 238 +238 1 238 +255 1 255 +255 1 255 +273 1 273 +273 1 273 +273 1 273 +278 1 278 +278 1 278 +311 1 311 +311 1 311 +311 1 311 +369 1 369 +369 1 369 +369 1 369 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +406 1 406 +406 1 406 +406 1 406 +406 1 406 +66 1 66 +98 1 98 +98 1 98 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + xx:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + yy + TableScan + alias: yy + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + value expressions: + expr: key + type: string + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN src yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 128 +128 1 128 +128 1 128 +146 1 146 +146 1 146 +150 1 150 +213 1 213 +213 1 213 +224 1 224 +224 1 224 +238 1 238 +238 1 238 +255 1 255 +255 1 255 +273 1 273 +273 1 273 +273 1 273 +278 1 278 +278 1 278 +311 1 311 +311 1 311 +311 1 311 +369 1 369 +369 1 369 +369 1 369 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +401 1 401 +406 1 406 +406 1 406 +406 1 406 +406 1 406 +66 1 66 +98 1 98 +98 1 98 +PREHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery xx and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_TABREF (TOK_TABNAME src) zz) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL zz) key))) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL zz) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-4 depends on stages: Stage-3 + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 2 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + zz + TableScan + alias: zz + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {VALUE._col0} + 1 + 2 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col8, _col9 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col8 + type: string + expr: _col9 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 128 9 +128 128 9 +128 128 9 +146 146 4 +146 146 4 +150 150 1 +213 213 4 +213 213 4 +224 224 4 +224 224 4 +238 238 4 +238 238 4 +255 255 4 +255 255 4 +273 273 9 +273 273 9 +273 273 9 +278 278 4 +278 278 4 +311 311 9 +311 311 9 +311 311 9 +369 369 9 +369 369 9 +369 369 9 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +406 406 16 +406 406 16 +406 406 16 +406 406 16 +66 66 1 +98 98 4 +98 98 4 +PREHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_TABREF (TOK_TABNAME src) zz) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL zz) key))) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL zz) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 3 + zz + TableScan + alias: zz + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Demux Operator + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {VALUE._col0} + 1 + 2 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col8, _col9 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col8 + type: string + expr: _col9 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {VALUE._col0} + 1 + 2 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col8, _col9 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col8 + type: string + expr: _col9 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN src zz ON xx.key=zz.key +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON zz.key=yy.key +ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 128 9 +128 128 9 +128 128 9 +146 146 4 +146 146 4 +150 150 1 +213 213 4 +213 213 4 +224 224 4 +224 224 4 +238 238 4 +238 238 4 +255 255 4 +255 255 4 +273 273 9 +273 273 9 +273 273 9 +278 278 4 +278 278 4 +311 311 9 +311 311 9 +311 311 9 +369 369 9 +369 369 9 +369 369 9 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +406 406 16 +406 406 16 +406 406 16 +406 406 16 +66 66 1 +98 98 4 +98 98 4 +PREHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 4 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery yy and xx join yy join zz. +EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key))) (TOK_TABREF (TOK_TABNAME src) zz) (= (. (TOK_TABLE_OR_COL yy) key) (. (TOK_TABLE_OR_COL zz) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-4 depends on stages: Stage-3 + Stage-1 depends on stages: Stage-4 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + zz + TableScan + alias: zz + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + 2 + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 128 9 +128 128 9 +128 128 9 +146 146 4 +146 146 4 +150 150 1 +213 213 4 +213 213 4 +224 224 4 +224 224 4 +238 238 4 +238 238 4 +255 255 4 +255 255 4 +273 273 9 +273 273 9 +273 273 9 +278 278 4 +278 278 4 +311 311 9 +311 311 9 +311 311 9 +369 369 9 +369 369 9 +369 369 9 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +406 406 16 +406 406 16 +406 406 16 +406 406 16 +66 66 1 +98 98 4 +98 98 4 +PREHOOK: query: EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key))) (TOK_TABREF (TOK_TABNAME src) zz) (= (. (TOK_TABLE_OR_COL yy) key) (. (TOK_TABLE_OR_COL zz) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx + TableScan + alias: xx + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + zz + TableScan + alias: zz + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 3 + Reduce Operator Tree: + Demux Operator + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + 2 + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + Inner Join 1 to 2 + condition expressions: + 0 {VALUE._col0} + 1 {VALUE._col0} {VALUE._col1} + 2 + handleSkewJoin: false + outputColumnNames: _col0, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col4 + type: string + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, yy.cnt +FROM src1 xx +JOIN +(SELECT x.key as key, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key) yy +ON xx.key=yy.key JOIN src zz +ON yy.key=zz.key ORDER BY xx.key, yy.key, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 128 9 +128 128 9 +128 128 9 +146 146 4 +146 146 4 +150 150 1 +213 213 4 +213 213 4 +224 224 4 +224 224 4 +238 238 4 +238 238 4 +255 255 4 +255 255 4 +273 273 9 +273 273 9 +273 273 9 +278 278 4 +278 278 4 +311 311 9 +311 311 9 +311 311 9 +369 369 9 +369 369 9 +369 369 9 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +401 401 25 +406 406 16 +406 406 16 +406 406 16 +406 406 16 +66 66 1 +98 98 4 +98 98 4 +PREHOOK: query: -- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery tmp and tmp join z. +EXPLAIN +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 2 MR jobs are needed. +-- The first job will evaluate subquery tmp and tmp join z. +EXPLAIN +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key) key) (TOK_SELEXPR (TOK_FUNCTION sum (. (TOK_TABLE_OR_COL xx) cnt)) sum1) (TOK_SELEXPR (TOK_FUNCTION sum (. (TOK_TABLE_OR_COL yy) cnt)) sum2)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL xx) key)))) tmp) (TOK_TABREF (TOK_TABNAME src) z) (= (. (TOK_TABLE_OR_COL tmp) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp) sum1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp) sum2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL tmp) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL tmp) sum1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL tmp) sum2)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL z) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL z) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-7 + Stage-3 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-3 + Stage-5 depends on stages: Stage-4 + Stage-7 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col3 + Group By Operator + aggregations: + expr: sum(_col1) + expr: sum(_col3) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0) + expr: sum(VALUE._col1) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + sort order: +++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + Reduce Operator Tree: + Extract + 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-7 + Map Reduce + Alias -> Map Operator Tree: + tmp:xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 3 1 128 val_128 +128 3 1 128 val_128 +128 3 1 128 val_128 +146 2 1 146 val_146 +146 2 1 146 val_146 +150 1 1 150 val_150 +213 2 1 213 val_213 +213 2 1 213 val_213 +224 2 1 224 val_224 +224 2 1 224 val_224 +238 2 1 238 val_238 +238 2 1 238 val_238 +255 2 1 255 val_255 +255 2 1 255 val_255 +273 3 1 273 val_273 +273 3 1 273 val_273 +273 3 1 273 val_273 +278 2 1 278 val_278 +278 2 1 278 val_278 +311 3 1 311 val_311 +311 3 1 311 val_311 +311 3 1 311 val_311 +369 3 1 369 val_369 +369 3 1 369 val_369 +369 3 1 369 val_369 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +406 4 1 406 val_406 +406 4 1 406 val_406 +406 4 1 406 val_406 +406 4 1 406 val_406 +66 1 1 66 val_66 +98 2 1 98 val_98 +98 2 1 98 val_98 +PREHOOK: query: EXPLAIN +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key) key) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key) key) (TOK_SELEXPR (TOK_FUNCTION sum (. (TOK_TABLE_OR_COL xx) cnt)) sum1) (TOK_SELEXPR (TOK_FUNCTION sum (. (TOK_TABLE_OR_COL yy) cnt)) sum2)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL xx) key)))) tmp) (TOK_TABREF (TOK_TABNAME src) z) (= (. (TOK_TABLE_OR_COL tmp) key) (. (TOK_TABLE_OR_COL z) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp) sum1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL tmp) sum2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL z) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL tmp) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL tmp) sum1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL tmp) sum2)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL z) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL z) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + tmp:xx:x + TableScan + alias: x + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + tmp:yy:y + TableScan + alias: y + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count() + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + z + TableScan + alias: z + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col3 + Mux Operator + Group By Operator + aggregations: + expr: sum(_col1) + expr: sum(_col3) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col3 + Mux Operator + Group By Operator + aggregations: + expr: sum(_col1) + expr: sum(_col3) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + sort order: +++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: bigint + expr: _col3 + type: string + expr: _col4 + type: string + Reduce Operator Tree: + Extract + 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 tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +FROM +(SELECT xx.key as key, sum(xx.cnt) as sum1, sum(yy.cnt) as sum2 + FROM (SELECT x.key as key, count(*) AS cnt FROM src x group by x.key) xx + JOIN (SELECT y.key as key, count(*) AS cnt FROM src1 y group by y.key) yy + ON (xx.key=yy.key) GROUP BY xx.key) tmp +JOIN src z ON tmp.key=z.key +ORDER BY tmp.key, tmp.sum1, tmp.sum2, z.key, z.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 3 1 128 val_128 +128 3 1 128 val_128 +128 3 1 128 val_128 +146 2 1 146 val_146 +146 2 1 146 val_146 +150 1 1 150 val_150 +213 2 1 213 val_213 +213 2 1 213 val_213 +224 2 1 224 val_224 +224 2 1 224 val_224 +238 2 1 238 val_238 +238 2 1 238 val_238 +255 2 1 255 val_255 +255 2 1 255 val_255 +273 3 1 273 val_273 +273 3 1 273 val_273 +273 3 1 273 val_273 +278 2 1 278 val_278 +278 2 1 278 val_278 +311 3 1 311 val_311 +311 3 1 311 val_311 +311 3 1 311 val_311 +369 3 1 369 val_369 +369 3 1 369 val_369 +369 3 1 369 val_369 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +401 5 1 401 val_401 +406 4 1 406 val_406 +406 4 1 406 val_406 +406 4 1 406 val_406 +406 4 1 406 val_406 +66 1 1 66 val_66 +98 2 1 98 val_98 +98 2 1 98 val_98 +PREHOOK: query: -- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 4 MR jobs are needed. +-- 2 MR jobs are used to evaluate yy, 1 MR is used to evaluate xx and xx join yy. +-- The last MR is used for ordering. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When Correlation Optimizer is turned off, 6 MR jobs are needed. +-- When Correlation Optimizer is turned on, 4 MR jobs are needed. +-- 2 MR jobs are used to evaluate yy, 1 MR is used to evaluate xx and xx join yy. +-- The last MR is used for ordering. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL x) value)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2, Stage-7 + Stage-4 depends on stages: Stage-3 + Stage-6 is a root stage + Stage-7 depends on stages: Stage-6 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + expr: _col1 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + sort order: +++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + Reduce Operator Tree: + Extract + 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-6 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + xx:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-7 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 128 val_128 9 +146 1 146 val_146 4 +150 1 150 val_150 1 +213 1 213 val_213 4 +224 1 224 val_224 4 +238 1 238 val_238 4 +255 1 255 val_255 4 +273 1 273 val_273 9 +278 1 278 val_278 4 +311 1 311 val_311 9 +369 1 369 val_369 9 +401 1 401 val_401 25 +406 1 406 val_406 16 +66 1 66 val_66 1 +98 1 98 val_98 4 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL x) value)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-2 + Stage-4 depends on stages: Stage-3 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + expr: value + type: string + yy:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 + handleSkewJoin: false + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + expr: _col1 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 2 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + xx:x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 0 + value expressions: + expr: key + type: string + xx:y + TableScan + alias: y + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + Reduce Operator Tree: + Demux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} + 1 + handleSkewJoin: false + outputColumnNames: _col0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Mux Operator + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: complete + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + sort order: +++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 128 val_128 9 +146 1 146 val_146 4 +150 1 150 val_150 1 +213 1 213 val_213 4 +224 1 224 val_224 4 +238 1 238 val_238 4 +255 1 255 val_255 4 +273 1 273 val_273 9 +278 1 278 val_278 4 +311 1 311 val_311 9 +369 1 369 val_369 9 +401 1 401 val_401 25 +406 1 406 val_406 16 +66 1 66 val_66 1 +98 1 98 val_98 4 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src1) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value) value) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL x) value)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-10 is a root stage + Stage-8 depends on stages: Stage-10 + Stage-3 depends on stages: Stage-8, Stage-9 + Stage-4 depends on stages: Stage-3 + Stage-11 is a root stage + Stage-9 depends on stages: Stage-11 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-10 + Map Reduce Local Work + Alias -> Map Local Tables: + yy:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + yy:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + yy:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0, _col1 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + outputColumnNames: _col0, _col1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + expr: _col1 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + sort order: +++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col4 + type: bigint + Reduce Operator Tree: + Extract + 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-11 + Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-9 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value, yy.cnt +FROM +(SELECT x.key as key, count(1) as cnt FROM src1 x JOIN src1 y ON (x.key = y.key) group by x.key) xx +JOIN +(SELECT x.key as key, x.value as value, count(1) as cnt FROM src x JOIN src y ON (x.key = y.key) group by x.key, x.value) yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 1 128 val_128 9 +146 1 146 val_146 4 +150 1 150 val_150 1 +213 1 213 val_213 4 +224 1 224 val_224 4 +238 1 238 val_238 4 +255 1 255 val_255 4 +273 1 273 val_273 9 +278 1 278 val_278 4 +311 1 311 val_311 9 +369 1 369 val_369 9 +401 1 401 val_401 25 +406 1 406 val_406 16 +66 1 66 val_66 1 +98 1 98 val_98 4 Index: ql/src/test/results/clientpositive/correlationoptimizer7.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer7.q.out @@ -0,0 +1,1048 @@ +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src1) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-9 is a root stage + Stage-7 depends on stages: Stage-9 + Stage-4 depends on stages: Stage-7 + Stage-8 depends on stages: Stage-4 + Stage-6 depends on stages: Stage-8 + Stage-2 depends on stages: Stage-6 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-9 + Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-7 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + yy + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + yy + TableScan + alias: yy + HashTable Sink Operator + condition expressions: + 0 {_col0} {_col1} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[_col0]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {_col0} {_col1} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[_col0]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 3 128 +146 2 146 val_146 +150 1 150 val_150 +213 2 213 val_213 +224 2 224 +238 2 238 val_238 +255 2 255 val_255 +273 3 273 val_273 +278 2 278 val_278 +311 3 311 val_311 +369 3 369 +401 5 401 val_401 +406 4 406 val_406 +66 1 66 val_66 +98 2 98 val_98 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src1) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-6 is a root stage + Stage-5 depends on stages: Stage-6 + Stage-1 depends on stages: Stage-5 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-6 + Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + yy + TableScan + alias: yy + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 3 128 +146 2 146 val_146 +150 1 150 val_150 +213 2 213 val_213 +224 2 224 +238 2 238 val_238 +255 2 255 val_255 +273 3 273 val_273 +278 2 278 val_278 +311 3 311 val_311 +369 3 369 +401 5 401 val_401 +406 4 406 val_406 +66 1 66 val_66 +98 2 98 val_98 +PREHOOK: query: -- Without correlation optimizer, we will have 3 MR jobs. +-- The first one is a MapJoin and Aggregation (in the Reduce Phase). +-- The second one is another MapJoin. The third one is for ordering. +-- With the correlation optimizer, right now, we still have +-- 3 MR jobs. The first one is a MapJoin and the map-side aggregation (a map-only job). +-- The second one have the reduce-side aggregation and the second join. +-- The third one is for ordering. +-- Although we have turned on hive.optimize.mapjoin.mapreduce, that optimizer +-- can not handle the case that the MR job (the one which a map-only job will be merged in) +-- has multiple inputs. We should improve that optimizer. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +PREHOOK: type: QUERY +POSTHOOK: query: -- Without correlation optimizer, we will have 3 MR jobs. +-- The first one is a MapJoin and Aggregation (in the Reduce Phase). +-- The second one is another MapJoin. The third one is for ordering. +-- With the correlation optimizer, right now, we still have +-- 3 MR jobs. The first one is a MapJoin and the map-side aggregation (a map-only job). +-- The second one have the reduce-side aggregation and the second join. +-- The third one is for ordering. +-- Although we have turned on hive.optimize.mapjoin.mapreduce, that optimizer +-- can not handle the case that the MR job (the one which a map-only job will be merged in) +-- has multiple inputs. We should improve that optimizer. +EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src1) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-9 is a root stage + Stage-7 depends on stages: Stage-9 + Stage-8 depends on stages: Stage-7 + Stage-6 depends on stages: Stage-8 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-9 + Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-7 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + yy + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + yy + TableScan + alias: yy + HashTable Sink Operator + condition expressions: + 0 {_col0} {_col1} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[_col0]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {_col0} {_col1} + 1 {key} {value} + handleSkewJoin: false + keys: + 0 [Column[_col0]] + 1 [Column[key]] + outputColumnNames: _col0, _col1, _col2, _col3 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 3 128 +146 2 146 val_146 +150 1 150 val_150 +213 2 213 val_213 +224 2 224 +238 2 238 val_238 +255 2 255 val_255 +273 3 273 val_273 +278 2 278 val_278 +311 3 311 val_311 +369 3 369 +401 5 401 val_401 +406 4 406 val_406 +66 1 66 val_66 +98 2 98 val_98 +PREHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) x) (TOK_TABREF (TOK_TABNAME src1) y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) xx) (TOK_TABREF (TOK_TABNAME src1) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) value))))) + +STAGE DEPENDENCIES: + Stage-6 is a root stage + Stage-5 depends on stages: Stage-6 + Stage-1 depends on stages: Stage-5 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-6 + Map Reduce Local Work + Alias -> Map Local Tables: + xx:y + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + xx:y + TableScan + alias: y + HashTable Sink Operator + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + Position of Big Table: 0 + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} + 1 + handleSkewJoin: false + keys: + 0 [Column[key]] + 1 [Column[key]] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: + expr: _col0 + type: string + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: _col0 + type: string + mode: hash + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + yy + TableScan + alias: yy + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + Reduce Operator Tree: + Extract + 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 xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, xx.cnt, yy.key, yy.value +FROM (SELECT x.key AS key, count(1) AS cnt + FROM src x JOIN src1 y ON (x.key = y.key) + GROUP BY x.key) xx +JOIN src1 yy +ON xx.key=yy.key ORDER BY xx.key, xx.cnt, yy.key, yy.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 3 128 +146 2 146 val_146 +150 1 150 val_150 +213 2 213 val_213 +224 2 224 +238 2 238 val_238 +255 2 255 val_255 +273 3 273 val_273 +278 2 278 val_278 +311 3 311 val_311 +369 3 369 +401 5 401 val_401 +406 4 406 val_406 +66 1 66 val_66 +98 2 98 val_98 Index: ql/src/test/results/clientpositive/correlationoptimizer8.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer8.q.out @@ -0,0 +1,1806 @@ +PREHOOK: query: -- When the Correlation Optimizer is turned off, this query will be evaluated by +-- 4 MR jobs. +-- When the Correlation Optimizer is turned on, because both inputs of the +-- UnionOperator are correlated, we can use 2 MR jobs to evaluate this query. +-- The first MR job will evaluate subquery subq1 and subq1 join x. The second +-- MR is for ordering. +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When the Correlation Optimizer is turned off, this query will be evaluated by +-- 4 MR jobs. +-- When the Correlation Optimizer is turned on, because both inputs of the +-- UnionOperator are correlated, we can use 2 MR jobs to evaluate this query. +-- The first MR job will evaluate subquery subq1 and subq1 join x. The second +-- MR is for ordering. +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) key) 20)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) key) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) key))))) subq1) (TOK_TABREF (TOK_TABNAME src1) x) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL subq1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + null-subquery1:subq1-subquery1:x + TableScan + alias: x + Filter Operator + predicate: + expr: (key < 20.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + TableScan + Union + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint +#### A masked pattern was here #### + TableScan + Union + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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-4 + Map Reduce + Alias -> Map Operator Tree: + null-subquery2:subq1-subquery2:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (key > 100.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 3 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 2 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 3 +401 val_401 5 +406 val_406 4 +PREHOOK: query: EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) key) 20)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) key) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) key))))) subq1) (TOK_TABREF (TOK_TABNAME src1) x) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL subq1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + null-subquery1:subq1-subquery1:x + TableScan + alias: x + Filter Operator + predicate: + expr: (key < 20.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + null-subquery2:subq1-subquery2:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (key > 100.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Union + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Union + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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 x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### +128 3 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 2 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 3 +401 val_401 5 +406 val_406 4 +PREHOOK: query: -- When the Correlation Optimizer is turned off, this query will be evaluated by +-- 4 MR jobs. +-- When the Correlation Optimizer is turned on, because both inputs of the +-- UnionOperator are correlated, we can use 2 MR jobs to evaluate this query. +-- The first MR job will evaluate subquery subq1 and subq1 join x. The second +-- MR is for ordering. +EXPLAIN +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value +PREHOOK: type: QUERY +POSTHOOK: query: -- When the Correlation Optimizer is turned off, this query will be evaluated by +-- 4 MR jobs. +-- When the Correlation Optimizer is turned on, because both inputs of the +-- UnionOperator are correlated, we can use 2 MR jobs to evaluate this query. +-- The first MR job will evaluate subquery subq1 and subq1 join x. The second +-- MR is for ordering. +EXPLAIN +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) key) 20)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) value) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) key) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) value))))) subq1) (TOK_TABREF (TOK_TABNAME src1) x) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL subq1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + null-subquery1:subq1-subquery1:x + TableScan + alias: x + Filter Operator + predicate: + expr: (key < 20.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + TableScan + Union + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint +#### A masked pattern was here #### + TableScan + Union + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + Reduce Operator Tree: + Extract + 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-4 + Map Reduce + Alias -> Map Operator Tree: + null-subquery2:subq1-subquery2:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (key > 100.0) + type: boolean + Select Operator + expressions: + expr: value + type: string + outputColumnNames: value + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: value + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### + 3 + 3 + 3 + 3 + 3 val_165 + 3 val_193 + 3 val_265 + 3 val_27 + 3 val_409 + 3 val_484 +0 3 NULL NULL +10 1 NULL NULL +11 1 NULL NULL +12 2 NULL NULL +15 2 NULL NULL +17 1 NULL NULL +18 2 NULL NULL +19 1 NULL NULL +2 1 NULL NULL +4 1 NULL NULL +5 3 NULL NULL +8 1 NULL NULL +9 1 NULL NULL +val_146 1 NULL NULL +val_150 1 NULL NULL +val_213 1 NULL NULL +val_238 1 NULL NULL +val_255 1 NULL NULL +val_273 1 NULL NULL +val_278 1 NULL NULL +val_311 1 NULL NULL +val_401 1 NULL NULL +val_406 1 NULL NULL +PREHOOK: query: EXPLAIN +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) key) 20)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) value) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) key) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) value))))) subq1) (TOK_TABREF (TOK_TABNAME src1) x) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL subq1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + null-subquery1:subq1-subquery1:x + TableScan + alias: x + Filter Operator + predicate: + expr: (key < 20.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + null-subquery2:subq1-subquery2:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (key > 100.0) + type: boolean + Select Operator + expressions: + expr: value + type: string + outputColumnNames: value + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: value + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 1 + value expressions: + expr: _col1 + type: bigint + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 2 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Union + Mux Operator + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Union + Mux Operator + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Mux Operator + Join Operator + condition map: + Left Outer Join0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + Reduce Operator Tree: + Extract + 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 subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Input: default@src1 +#### A masked pattern was here #### +POSTHOOK: query: SELECT subq1.key, subq1.cnt, x.key, x.value +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.value as key, count(1) as cnt from src1 x1 where x1.key > 100 group by x1.value +) subq1 +LEFT OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.cnt, x.key, x.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Input: default@src1 +#### A masked pattern was here #### + 3 + 3 + 3 + 3 + 3 val_165 + 3 val_193 + 3 val_265 + 3 val_27 + 3 val_409 + 3 val_484 +0 3 NULL NULL +10 1 NULL NULL +11 1 NULL NULL +12 2 NULL NULL +15 2 NULL NULL +17 1 NULL NULL +18 2 NULL NULL +19 1 NULL NULL +2 1 NULL NULL +4 1 NULL NULL +5 3 NULL NULL +8 1 NULL NULL +9 1 NULL NULL +val_146 1 NULL NULL +val_150 1 NULL NULL +val_213 1 NULL NULL +val_238 1 NULL NULL +val_255 1 NULL NULL +val_273 1 NULL NULL +val_278 1 NULL NULL +val_311 1 NULL NULL +val_401 1 NULL NULL +val_406 1 NULL NULL +PREHOOK: query: -- When the Correlation Optimizer is turned on, because a input of UnionOperator is +-- not correlated, we cannot handle this case right now. So, this query will not be +-- optimized. +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key, x1.value +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- When the Correlation Optimizer is turned on, because a input of UnionOperator is +-- not correlated, we cannot handle this case right now. So, this query will not be +-- optimized. +EXPLAIN +SELECT x.key, x.value, subq1.cnt +FROM +( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key + UNION ALL + SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key, x1.value +) subq1 +JOIN src1 x ON (x.key = subq1.key) ORDER BY x.key, x.value, subq1.cnt +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) key) 20)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) key) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) key) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) key) (. (TOK_TABLE_OR_COL x1) value))))) subq1) (TOK_TABREF (TOK_TABNAME src1) x) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL subq1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + null-subquery1:subq1-subquery1:x + TableScan + alias: x + Filter Operator + predicate: + expr: (key < 20.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + TableScan + Union + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint +#### A masked pattern was here #### + TableScan + Union + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: 0 + value expressions: + expr: _col1 + type: bigint + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: key + type: string + sort order: + + Map-reduce partition columns: + expr: key + type: string + tag: 1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col1, _col2, _col3 + Select Operator + expressions: + expr: _col2 + type: string + expr: _col3 + type: string + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + sort order: +++ + tag: -1 + value expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Extract + 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-4 + Map Reduce + Alias -> Map Operator Tree: + null-subquery2:subq1-subquery2:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (key > 100.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + expr: value + type: string + outputColumnNames: key, value + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + expr: value + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: string + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: -- When the Correlation Optimizer is turned on, because a input of UnionOperator is +-- not correlated, we cannot handle this case right now. So, this query will not be +-- optimized. +EXPLAIN +SELECT subq1.key, subq1.value, x.key, x.value +FROM +( SELECT cast(x.key as INT) as key, count(1) as value from src x where x.key < 20 group by x.key + UNION ALL + SELECT count(1) as key, cast(x1.key as INT) as value from src x1 where x1.key > 100 group by x1.key +) subq1 +FULL OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.value, x.key, x.value +PREHOOK: type: QUERY +POSTHOOK: query: -- When the Correlation Optimizer is turned on, because a input of UnionOperator is +-- not correlated, we cannot handle this case right now. So, this query will not be +-- optimized. +EXPLAIN +SELECT subq1.key, subq1.value, x.key, x.value +FROM +( SELECT cast(x.key as INT) as key, count(1) as value from src x where x.key < 20 group by x.key + UNION ALL + SELECT count(1) as key, cast(x1.key as INT) as value from src x1 where x1.key > 100 group by x1.key +) subq1 +FULL OUTER JOIN src1 x ON (x.key = subq1.key) ORDER BY subq1.key, subq1.value, x.key, x.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_FULLOUTERJOIN (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION TOK_INT (. (TOK_TABLE_OR_COL x) key)) key) (TOK_SELEXPR (TOK_FUNCTION count 1) value)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) key) 20)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION count 1) key) (TOK_SELEXPR (TOK_FUNCTION TOK_INT (. (TOK_TABLE_OR_COL x1) key)) value)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) key) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) key))))) subq1) (TOK_TABREF (TOK_TABNAME src1) x) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL subq1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL subq1) value)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL x) value))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + null-subquery1:subq1-subquery1:x + TableScan + alias: x + Filter Operator + predicate: + expr: (key < 20.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: UDFToLong(UDFToInteger(_col0)) + type: bigint + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + TableScan + Union + Reduce Output Operator + key expressions: + expr: UDFToDouble(_col0) + type: double + sort order: + + Map-reduce partition columns: + expr: UDFToDouble(_col0) + type: double + tag: 0 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint +#### A masked pattern was here #### + TableScan + Union + Reduce Output Operator + key expressions: + expr: UDFToDouble(_col0) + type: double + sort order: + + Map-reduce partition columns: + expr: UDFToDouble(_col0) + type: double + tag: 0 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + x + TableScan + alias: x + Reduce Output Operator + key expressions: + expr: UDFToDouble(key) + type: double + sort order: + + Map-reduce partition columns: + expr: UDFToDouble(key) + type: double + tag: 1 + value expressions: + expr: key + type: string + expr: value + type: string + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: bigint + expr: _col1 + type: bigint + expr: _col2 + type: string + expr: _col3 + type: string + Reduce Operator Tree: + Extract + 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-4 + Map Reduce + Alias -> Map Operator Tree: + null-subquery2:subq1-subquery2:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (key > 100.0) + type: boolean + Select Operator + expressions: + expr: key + type: string + outputColumnNames: key + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: key + type: string + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: string + sort order: + + Map-reduce partition columns: + expr: _col0 + type: string + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col1 + type: bigint + expr: UDFToLong(UDFToInteger(_col0)) + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + Index: ql/src/test/results/clientpositive/correlationoptimizer9.q.out =================================================================== --- /dev/null +++ ql/src/test/results/clientpositive/correlationoptimizer9.q.out @@ -0,0 +1,1181 @@ +PREHOOK: query: CREATE TABLE tmp(c1 INT, c2 INT, c3 STRING, c4 STRING) +PREHOOK: type: CREATETABLE +POSTHOOK: query: CREATE TABLE tmp(c1 INT, c2 INT, c3 STRING, c4 STRING) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@tmp +PREHOOK: query: INSERT OVERWRITE TABLE tmp +SELECT x.key, y.key, x.value, y.value FROM src x JOIN src y ON (x.key = y.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@tmp +POSTHOOK: query: INSERT OVERWRITE TABLE tmp +SELECT x.key, y.key, x.value, y.value FROM src x JOIN src y ON (x.key = y.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@tmp +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: EXPLAIN +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) c1) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) c1) 120)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) c1)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) c2) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) c2) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) c2)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (c2 > 100) + type: boolean + Select Operator + expressions: + expr: c2 + type: int + outputColumnNames: c2 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c2 + type: int + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: 1 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: 0 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col2 + type: int + expr: _col1 + type: bigint + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: int + expr: _col2 + type: bigint + expr: _col3 + type: bigint + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: int + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Extract + 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-4 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Filter Operator + predicate: + expr: (c1 < 120) + type: boolean + Select Operator + expressions: + expr: c1 + type: int + outputColumnNames: c1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c1 + type: int + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: -1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@tmp +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tmp +#### A masked pattern was here #### +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +103 103 4 4 +104 104 4 4 +105 105 1 1 +111 111 1 1 +113 113 4 4 +114 114 1 1 +116 116 1 1 +118 118 4 4 +119 119 9 9 +PREHOOK: query: -- The merged table scan should be able to load both c1 and c2 +EXPLAIN +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: -- The merged table scan should be able to load both c1 and c2 +EXPLAIN +SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) c1) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) c1) 120)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) c1)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) c2) key) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) c2) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) c2)))) yy) (= (. (TOK_TABLE_OR_COL xx) key) (. (TOK_TABLE_OR_COL yy) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Filter Operator + predicate: + expr: (c1 < 120) + type: boolean + Select Operator + expressions: + expr: c1 + type: int + outputColumnNames: c1 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c1 + type: int + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: 0 + value expressions: + expr: _col1 + type: bigint + yy:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (c2 > 100) + type: boolean + Select Operator + expressions: + expr: c2 + type: int + outputColumnNames: c2 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c2 + type: int + mode: hash + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + sort order: + + Map-reduce partition columns: + expr: _col0 + type: int + tag: 1 + value expressions: + expr: _col1 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col2 + type: int + expr: _col1 + type: bigint + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: bigint + outputColumnNames: _col0, _col1 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} + 1 {VALUE._col0} {VALUE._col1} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col2 + type: int + expr: _col1 + type: bigint + expr: _col3 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: int + expr: _col2 + type: bigint + expr: _col3 + type: bigint + sort order: ++++ + tag: -1 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: int + expr: _col2 + type: bigint + expr: _col3 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@tmp +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key, yy.key, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1) xx +JOIN +(SELECT x1.c2 AS key, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c2) yy +ON (xx.key = yy.key) ORDER BY xx.key, yy.key, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tmp +#### A masked pattern was here #### +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +103 103 4 4 +104 104 4 4 +105 105 1 1 +111 111 1 1 +113 113 4 4 +114 114 1 1 +116 116 1 1 +118 118 4 4 +119 119 9 9 +PREHOOK: query: EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) c1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) c3) key2) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) c1) 120)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) c1) (. (TOK_TABLE_OR_COL x) c3)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) c1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) c3) key2) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) c2) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) c1) (. (TOK_TABLE_OR_COL x1) c3)))) yy) (AND (= (. (TOK_TABLE_OR_COL xx) key1) (. (TOK_TABLE_OR_COL yy) key1)) (== (. (TOK_TABLE_OR_COL xx) key2) (. (TOK_TABLE_OR_COL yy) key2))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key2)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key2)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1, Stage-4 + Stage-3 depends on stages: Stage-2 + Stage-4 is a root stage + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + yy:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (c2 > 100) + type: boolean + Select Operator + expressions: + expr: c1 + type: int + expr: c3 + type: string + outputColumnNames: c1, c3 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c1 + type: int + expr: c3 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: int + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: int + expr: _col1 + type: string + tag: 1 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: bigint + $INTNAME1 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: int + expr: _col1 + type: string + tag: 0 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: bigint + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col3 + type: int + expr: _col4 + type: string + expr: _col2 + type: bigint + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: bigint + expr: _col5 + type: bigint + sort order: ++++++ + tag: -1 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: bigint + expr: _col5 + type: bigint + Reduce Operator Tree: + Extract + 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-4 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Filter Operator + predicate: + expr: (c1 < 120) + type: boolean + Select Operator + expressions: + expr: c1 + type: int + expr: c3 + type: string + outputColumnNames: c1, c3 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c1 + type: int + expr: c3 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: int + expr: _col1 + type: string + tag: -1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@tmp +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tmp +#### A masked pattern was here #### +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +103 val_103 103 val_103 4 4 +104 val_104 104 val_104 4 4 +105 val_105 105 val_105 1 1 +111 val_111 111 val_111 1 1 +113 val_113 113 val_113 4 4 +114 val_114 114 val_114 1 1 +116 val_116 116 val_116 1 1 +118 val_118 118 val_118 4 4 +119 val_119 119 val_119 9 9 +PREHOOK: query: EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) c1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) c3) key2) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL x) c1) 120)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) c1) (. (TOK_TABLE_OR_COL x) c3)))) xx) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME tmp) x1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) c1) key1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL x1) c3) key2) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL x1) c2) 100)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x1) c1) (. (TOK_TABLE_OR_COL x1) c3)))) yy) (AND (= (. (TOK_TABLE_OR_COL xx) key1) (. (TOK_TABLE_OR_COL yy) key1)) (== (. (TOK_TABLE_OR_COL xx) key2) (. (TOK_TABLE_OR_COL yy) key2))))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) key2)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL yy) cnt))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) key2)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key1)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) key2)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL xx) cnt)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL yy) cnt))))) + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + xx:x + TableScan + alias: x + Filter Operator + predicate: + expr: (c1 < 120) + type: boolean + Select Operator + expressions: + expr: c1 + type: int + expr: c3 + type: string + outputColumnNames: c1, c3 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c1 + type: int + expr: c3 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: int + expr: _col1 + type: string + tag: 0 + value expressions: + expr: _col2 + type: bigint + yy:x1 + TableScan + alias: x1 + Filter Operator + predicate: + expr: (c2 > 100) + type: boolean + Select Operator + expressions: + expr: c1 + type: int + expr: c3 + type: string + outputColumnNames: c1, c3 + Group By Operator + aggregations: + expr: count(1) + bucketGroup: false + keys: + expr: c1 + type: int + expr: c3 + type: string + mode: hash + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + sort order: ++ + Map-reduce partition columns: + expr: _col0 + type: int + expr: _col1 + type: string + tag: 1 + value expressions: + expr: _col2 + type: bigint + Reduce Operator Tree: + Demux Operator + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col3 + type: int + expr: _col4 + type: string + expr: _col2 + type: bigint + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Group By Operator + aggregations: + expr: count(VALUE._col0) + bucketGroup: false + keys: + expr: KEY._col0 + type: int + expr: KEY._col1 + type: string + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: bigint + outputColumnNames: _col0, _col1, _col2 + Mux Operator + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {VALUE._col1} {VALUE._col2} + 1 {VALUE._col0} {VALUE._col1} {VALUE._col2} + handleSkewJoin: false + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Select Operator + expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col3 + type: int + expr: _col4 + type: string + expr: _col2 + type: bigint + expr: _col5 + type: bigint + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: +#### A masked pattern was here #### + Reduce Output Operator + key expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: bigint + expr: _col5 + type: bigint + sort order: ++++++ + tag: -1 + value expressions: + expr: _col0 + type: int + expr: _col1 + type: string + expr: _col2 + type: int + expr: _col3 + type: string + expr: _col4 + type: bigint + expr: _col5 + type: bigint + Reduce Operator Tree: + Extract + 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 xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +PREHOOK: type: QUERY +PREHOOK: Input: default@tmp +#### A masked pattern was here #### +POSTHOOK: query: SELECT xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +FROM +(SELECT x.c1 AS key1, x.c3 AS key2, count(1) AS cnt FROM tmp x WHERE x.c1 < 120 GROUP BY x.c1, x.c3) xx +JOIN +(SELECT x1.c1 AS key1, x1.c3 AS key2, count(1) AS cnt FROM tmp x1 WHERE x1.c2 > 100 GROUP BY x1.c1, x1.c3) yy +ON (xx.key1 = yy.key1 AND xx.key2 == yy.key2) ORDER BY xx.key1, xx.key2, yy.key1, yy.key2, xx.cnt, yy.cnt +POSTHOOK: type: QUERY +POSTHOOK: Input: default@tmp +#### A masked pattern was here #### +POSTHOOK: Lineage: tmp.c1 EXPRESSION [(src)x.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c2 EXPRESSION [(src)y.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c3 SIMPLE [(src)x.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: tmp.c4 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ] +103 val_103 103 val_103 4 4 +104 val_104 104 val_104 4 4 +105 val_105 105 val_105 1 1 +111 val_111 111 val_111 1 1 +113 val_113 113 val_113 4 4 +114 val_114 114 val_114 1 1 +116 val_116 116 val_116 1 1 +118 val_118 118 val_118 4 4 +119 val_119 119 val_119 9 9 Index: ql/src/test/results/compiler/plan/groupby2.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby2.q.xml +++ ql/src/test/results/compiler/plan/groupby2.q.xml @@ -193,6 +193,20 @@ + + KEY._col1:0._col0 + + + _col1 + + + + + + + + + @@ -214,17 +228,7 @@ - - - _col1 - - - - - - - - + @@ -1501,7 +1505,7 @@ _col1 - + _col1 @@ -1515,7 +1519,7 @@ _col0 - + _col0 @@ -1534,10 +1538,10 @@ - + - + @@ -1640,7 +1644,7 @@ _col0 - + KEY._col0 @@ -1727,7 +1731,7 @@ - + Index: ql/src/test/results/compiler/plan/groupby3.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby3.q.xml +++ ql/src/test/results/compiler/plan/groupby3.q.xml @@ -174,7 +174,26 @@ - + + + KEY._col0:0._col0 + + + _col0 + + + + + + + + string + + + + + + @@ -192,21 +211,7 @@ - - - _col0 - - - - - - - - string - - - - + @@ -1675,7 +1680,7 @@ _col4 - + _col4 @@ -1689,7 +1694,7 @@ _col3 - + _col3 @@ -1703,7 +1708,7 @@ _col2 - + _col2 @@ -1717,7 +1722,7 @@ _col1 - + _col1 @@ -1731,7 +1736,7 @@ _col0 - + _col0 @@ -1750,6 +1755,9 @@ + + + @@ -1761,9 +1769,6 @@ - - -