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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc>) 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 extends Serializable> 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 extends Serializable> 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 extends Serializable> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> child,
+ List childKeyCols, List childPartitionCols,
+ String childRSOrder,
+ Operator extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> op = (Operator extends OperatorDesc>) 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 extends Serializable> 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 extends OperatorDesc> startPoint) {
+ Operator extends OperatorDesc> 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 extends OperatorDesc> child = getSingleChild(op, true);
+ for (Operator extends OperatorDesc> 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 extends OperatorDesc> op)
+ throws SemanticException {
+ Operator extends OperatorDesc> 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 extends Serializable> 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 extends Serializable> 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 extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> demuxOp = OperatorFactory.get(demuxDesc);
+ demuxOp.setChildOperators(childrenOfDemux);
+ demuxOp.setParentOperators(parentRSsOfDemux);
+ for (Operator extends OperatorDesc> child: childrenOfDemux) {
+ List> parentsWithMultipleDemux =
+ new ArrayList>();
+ boolean hasBottomReduceSinkOperators = false;
+ boolean hasNonBottomReduceSinkOperators = false;
+ for (int i = 0; i < child.getParentOperators().size(); i++) {
+ Operator extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> childOP =
+ CorrelationUtilities.getSingleChild(rsop, true);
+ if (childOP instanceof GroupByOperator) {
+ CorrelationUtilities.removeReduceSinkForGroupBy(
+ rsop, (GroupByOperator)childOP, pCtx, corrCtx);
+ List> parentsOfMux =
+ new ArrayList>();
+ Operator extends OperatorDesc> parentOp =
+ CorrelationUtilities.getSingleParent(childOP, true);
+ parentsOfMux.add(parentOp);
+ Operator extends OperatorDesc> 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 extends OperatorDesc> 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 extends OperatorDesc> mux = OperatorFactory.get(muxDesc);
+ mux.setChildOperators(Utilities.makeList(childOP));
+ mux.setParentOperators(parentsOfMux);
+
+ for (Operator extends OperatorDesc> 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 extends Serializable> 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