diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 5efa98a..c2e78cc 100644
--- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -519,6 +519,7 @@
HIVEOPTBUCKETMAPJOIN("hive.optimize.bucketmapjoin", false), // optimize bucket map join
HIVEOPTSORTMERGEBUCKETMAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false), // try to use sorted merge bucket map join
HIVEOPTREDUCEDEDUPLICATION("hive.optimize.reducededuplication", true),
+ HIVEOPTREDUCEDEDUPLICATIONMINREDUCER("hive.optimize.reducededuplication.min.reducer", 4),
// whether to optimize union followed by select followed by filesink
// It creates sub-directories in the final output, so should not be turned on in systems
// where MAPREDUCE-1501 is not present
diff --git conf/hive-default.xml.template conf/hive-default.xml.template
index a005f39..3b5a1dc 100644
--- conf/hive-default.xml.template
+++ conf/hive-default.xml.template
@@ -1031,6 +1031,14 @@
+ hive.optimize.reducededuplication.min.reducer
+ 4
+ Reduce deduplication merges two RSs by moving key/parts/reducer-num of the child RS to parent RS.
+ That means if reducer-num of the child RS is fixed (order by or forced bucketing) and small, it can make very slow, single MR.
+ The optimization will be disabled if number of reducers is less than specified value.
+
+
+
hive.exec.dynamic.partition
true
Whether or not to allow dynamic partitions in DML/DDL.
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
index a9a93ad..b347e76 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
@@ -554,6 +554,7 @@ private static void pruneReduceSinkOperator(boolean[] retainFlags,
for (ColumnInfo colInfo : oldSchema) {
if (colInfo.getInternalName().startsWith(Utilities.ReduceField.KEY.toString() + ".")) {
String[] nm = oldRR.reverseLookup(colInfo.getInternalName());
+ newMap.put(colInfo.getInternalName(), oldMap.get(colInfo.getInternalName()));
newRR.put(nm[0], nm[1], colInfo);
sig.add(colInfo);
} else {
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkDeDuplication.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkDeDuplication.java
index 77c50b0..033d3d1 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkDeDuplication.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkDeDuplication.java
@@ -18,15 +18,22 @@
package org.apache.hadoop.hive.ql.optimizer;
+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;
+
+import java.io.Serializable;
+import java.lang.reflect.Array;
import java.util.ArrayList;
import java.util.HashMap;
-import java.util.Iterator;
+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.conf.HiveConf;
import org.apache.hadoop.hive.ql.exec.ColumnInfo;
import org.apache.hadoop.hive.ql.exec.ExtractOperator;
import org.apache.hadoop.hive.ql.exec.FilterOperator;
@@ -52,15 +59,22 @@
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.ExprNodeColumnDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
-import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+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 org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
/**
- * If two reducer sink operators share the same partition/sort columns, we
- * should merge them. This should happen after map join optimization because map
+ * 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.
*/
public class ReduceSinkDeDuplication implements Transform{
@@ -71,13 +85,31 @@
public ParseContext transform(ParseContext pctx) throws SemanticException {
pGraphContext = pctx;
- // generate pruned column list for all relevant operators
+ // generate pruned column list for all relevant operators
ReduceSinkDeduplicateProcCtx cppCtx = new ReduceSinkDeduplicateProcCtx(pGraphContext);
+ boolean mergeJoins = !pctx.getConf().getBoolVar(HIVECONVERTJOIN) &&
+ !pctx.getConf().getBoolVar(HIVECONVERTJOINNOCONDITIONALTASK);
+
Map opRules = new LinkedHashMap();
opRules.put(new RuleRegExp("R1",
- ReduceSinkOperator.getOperatorName() + "%.*" + ReduceSinkOperator.getOperatorName() + "%"),
- ReduceSinkDeduplicateProcFactory.getReducerReducerProc());
+ ReduceSinkOperator.getOperatorName() + "%"
+ + ".*%"
+ + ReduceSinkOperator.getOperatorName() + "%"),
+ ReduceSinkDeduplicateProcFactory.getReducerReducerProc());
+ opRules.put(new RuleRegExp("R2",
+ ReduceSinkOperator.getOperatorName() + "%"
+ + GroupByOperator.getOperatorName() + "%"
+ + ".*%"
+ + ReduceSinkOperator.getOperatorName() + "%"),
+ ReduceSinkDeduplicateProcFactory.getGroupbyReducerProc());
+ if (mergeJoins) {
+ opRules.put(new RuleRegExp("R3",
+ JoinOperator.getOperatorName() + "%"
+ + ".*%"
+ + ReduceSinkOperator.getOperatorName() + "%"),
+ ReduceSinkDeduplicateProcFactory.getJoinReducerProc());
+ }
// The dispatcher fires the processor corresponding to the closest matching
// rule and passes the context along
@@ -92,23 +124,27 @@ public ParseContext transform(ParseContext pctx) throws SemanticException {
return pGraphContext;
}
- class ReduceSinkDeduplicateProcCtx implements NodeProcessorCtx{
+ class ReduceSinkDeduplicateProcCtx implements NodeProcessorCtx {
+
ParseContext pctx;
- List rejectedRSList;
+ boolean trustScript;
+ // min reducer num for merged RS (to avoid query contains "order by" executed by one reducer)
+ int minReducer;
+ Set> removedOps;
public ReduceSinkDeduplicateProcCtx(ParseContext pctx) {
- rejectedRSList = new ArrayList();
+ removedOps = new HashSet>();
+ trustScript = pctx.getConf().getBoolVar(HIVESCRIPTOPERATORTRUST);
+ minReducer = pctx.getConf().getIntVar(HIVEOPTREDUCEDEDUPLICATIONMINREDUCER);
this.pctx = pctx;
}
- public boolean contains (ReduceSinkOperator rsOp) {
- return rejectedRSList.contains(rsOp);
+ public boolean contains(Operator> rsOp) {
+ return removedOps.contains(rsOp);
}
- public void addRejectedReduceSinkOperator(ReduceSinkOperator rsOp) {
- if (!rejectedRSList.contains(rsOp)) {
- rejectedRSList.add(rsOp);
- }
+ public boolean addRemovedOperator(Operator> rsOp) {
+ return removedOps.add(rsOp);
}
public ParseContext getPctx() {
@@ -120,351 +156,678 @@ public void setPctx(ParseContext 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;
+ /*
+ * 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 {
+
+ 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;
}
- static class ReducerReducerProc implements NodeProcessor {
- @Override
- public Object process(Node nd, Stack stack,
- NodeProcessorCtx procCtx, Object... nodeOutputs)
- throws SemanticException {
- ReduceSinkDeduplicateProcCtx ctx = (ReduceSinkDeduplicateProcCtx) procCtx;
- ReduceSinkOperator childReduceSink = (ReduceSinkOperator)nd;
+ protected abstract Object process(ReduceSinkOperator cRS, ParseContext context)
+ throws SemanticException;
- if(ctx.contains(childReduceSink)) {
- return null;
+ 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) {
+ 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 = containsExpr(cexpr, pexprs, cRS, pRSs);
+ if (found < 0 || !sorted[found]) {
+ 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 = containsExpr(cexpr, pexprs, cRS, pRSs);
+ if (found < 0 || !sorted[found]) {
+ return false;
+ }
+ }
- List> childOp =
- childReduceSink.getChildOperators();
- if (childOp != null && childOp.size() == 1) {
- Operator extends OperatorDesc> child = childOp.get(0);
- if (child instanceof GroupByOperator || child instanceof JoinOperator) {
- ctx.addRejectedReduceSinkOperator(childReduceSink);
- return null;
- }
+ if (moveReducerNumTo > 0) {
+ for (ReduceSinkOperator pRS : pRSs) {
+ pRS.getConf().setNumReducers(cRS.getConf().getNumReducers());
}
+ }
+ return true;
+ }
- ParseContext pGraphContext = ctx.getPctx();
- HashMap childColumnMapping =
- getPartitionAndKeyColumnMapping(childReduceSink);
- ReduceSinkOperator parentRS = null;
- parentRS = findSingleParentReduceSink(childReduceSink, pGraphContext);
- if (parentRS == null) {
- ctx.addRejectedReduceSinkOperator(childReduceSink);
- return null;
+ 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;
+ }
+
+ 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;
+ }
}
- HashMap parentColumnMapping = getPartitionAndKeyColumnMapping(parentRS);
- Operator extends OperatorDesc> stopBacktrackFlagOp = null;
- if (parentRS.getParentOperators() == null
- || parentRS.getParentOperators().size() == 0) {
- stopBacktrackFlagOp = parentRS;
- } else if (parentRS.getParentOperators().size() != 1) {
- return null;
+ }
+ return true;
+ }
+
+ protected boolean merge(ReduceSinkOperator cRS, ReduceSinkOperator pRS, int minReducer) {
+ int[] result = checkStatus(cRS, pRS, minReducer);
+ if (result == null) {
+ return false;
+ }
+ if (result[0] > 0) {
+ Operator> terminal = getSingleParent(pRS);
+ ArrayList childKCs = cRS.getConf().getKeyCols();
+ pRS.getConf().setKeyCols(backtrack(childKCs, cRS, terminal, 0));
+ }
+ if (result[1] > 0) {
+ Operator> terminal = getSingleParent(pRS);
+ ArrayList childPCs = cRS.getConf().getPartitionCols();
+ pRS.getConf().setPartitionCols(backtrack(childPCs, cRS, terminal, 0));
+ }
+ if (result[2] > 0) {
+ pRS.getConf().setOrder(cRS.getConf().getOrder());
+ }
+ if (result[3] > 0) {
+ pRS.getConf().setNumReducers(cRS.getConf().getNumReducers());
+ }
+ return true;
+ }
+
+ // -1 for p to c, 1 for c to p
+ private int[] checkStatus(ReduceSinkOperator cRS, ReduceSinkOperator pRS, int minReducer) {
+ 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, 0);
+ if (moveKeyColTo == null) {
+ return null;
+ }
+ List cpars = cConf.getPartitionCols();
+ List ppars = pConf.getPartitionCols();
+ Integer movePartitionColTo = checkExprs(cpars, ppars, cRS, pRS, 0);
+ if (movePartitionColTo == null) {
+ return null;
+ }
+ return new int[] {moveKeyColTo, movePartitionColTo, moveRSOrderTo, moveReducerNumTo};
+ }
+
+ private Integer checkExprs(List ckeys, List pkeys,
+ ReduceSinkOperator cRS, ReduceSinkOperator pRS, int tag) {
+ Integer moveKeyColTo = 0;
+ if (ckeys == null || ckeys.isEmpty()) {
+ if (pkeys != null && !pkeys.isEmpty()) {
+ Operator> terminal = getSingleParent(pRS);
+ for (ExprNodeDesc pkey : pkeys) {
+ if (backtrack(pkey, pRS, terminal, tag) == null) {
+ return null;
+ }
+ }
+ moveKeyColTo = -1;
+ }
+ } else {
+ if (pkeys == null || pkeys.isEmpty()) {
+ Operator> terminal = getSingleParent(pRS);
+ for (ExprNodeDesc ckey : ckeys) {
+ if (backtrack(ckey, cRS, terminal, tag) == null) {
+ return null;
+ }
+ }
+ moveKeyColTo = 1;
} else {
- stopBacktrackFlagOp = parentRS.getParentOperators().get(0);
+ moveKeyColTo = sameKeys(ckeys, pkeys, cRS, pRS, tag);
}
+ }
+ return moveKeyColTo;
+ }
- boolean succeed = backTrackColumnNames(childColumnMapping, childReduceSink, stopBacktrackFlagOp, pGraphContext);
- if (!succeed) {
+ protected Integer sameKeys(List cexprs, List pexprs,
+ Operator> child, Operator> parent, int tag) {
+ int common = Math.min(cexprs.size(), pexprs.size());
+ int limit = Math.max(cexprs.size(), pexprs.size());
+ Operator> terminal = getSingleParent(parent);
+ int i = 0;
+ for (; i < common; i++) {
+ ExprNodeDesc cexpr = backtrack(cexprs.get(i), child, terminal, tag);
+ ExprNodeDesc pexpr = backtrack(pexprs.get(i), parent, terminal, tag);
+ if (cexpr == null || !cexpr.isSame(pexpr)) {
return null;
}
- succeed = backTrackColumnNames(parentColumnMapping, parentRS, stopBacktrackFlagOp, pGraphContext);
- if (!succeed) {
+ }
+ for (;i < limit; i++) {
+ if (cexprs.size() > pexprs.size()) {
+ if (backtrack(cexprs.get(i), child, terminal, tag) == null) {
+ return null;
+ }
+ } else if (backtrack(pexprs.get(i), parent, terminal, tag) == null) {
return null;
}
+ }
+ return Integer.valueOf(cexprs.size()).compareTo(pexprs.size());
+ }
- boolean same = compareReduceSink(childReduceSink, parentRS, childColumnMapping, parentColumnMapping);
- if (!same) {
- return null;
+ protected int containsExpr(ExprNodeDesc cexpr, ExprNodeDesc[] pexprs,
+ Operator child, Operator[] parents) {
+ for (int tag = 0; tag < parents.length; tag++) {
+ if (sameKey(cexpr, pexprs[tag], child, parents[tag], tag)) {
+ return tag;
}
- replaceReduceSinkWithSelectOperator(childReduceSink, pGraphContext);
- return null;
}
+ return -1;
+ }
- private void replaceReduceSinkWithSelectOperator(
- ReduceSinkOperator childReduceSink, ParseContext pGraphContext) throws SemanticException {
- List> parentOp =
- childReduceSink.getParentOperators();
- List> childOp =
- childReduceSink.getChildOperators();
-
- Operator extends OperatorDesc> oldParent = childReduceSink;
+ protected boolean sameKey(ExprNodeDesc cexpr, ExprNodeDesc pexpr,
+ Operator child, Operator parent, int tag) {
+ Operator terminal = getSingleParent(parent);
+ ExprNodeDesc cexprb = backtrack(cexpr, child, terminal, tag);
+ ExprNodeDesc pexprb = backtrack(pexpr, parent, terminal, tag);
+ return cexprb != null && cexprb.isSame(pexprb);
+ }
- if (childOp != null && childOp.size() == 1
- && ((childOp.get(0)) instanceof ExtractOperator)) {
- oldParent = childOp.get(0);
- childOp = childOp.get(0).getChildOperators();
+ 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());
+ }
- Operator extends OperatorDesc> input = parentOp.get(0);
- input.getChildOperators().clear();
-
- RowResolver inputRR = pGraphContext.getOpParseCtx().get(input).getRowResolver();
-
- ArrayList exprs = new ArrayList();
- ArrayList outputs = new ArrayList();
- List outputCols = childReduceSink.getConf().getOutputValueColumnNames();
- RowResolver outputRS = new RowResolver();
-
- Map colExprMap = new HashMap();
-
- for (int i = 0; i < outputCols.size(); i++) {
- String internalName = outputCols.get(i);
- String[] nm = inputRR.reverseLookup(internalName);
- ColumnInfo valueInfo = inputRR.get(nm[0], nm[1]);
- ExprNodeDesc colDesc = childReduceSink.getConf().getValueCols().get(i);
- exprs.add(colDesc);
- outputs.add(internalName);
- outputRS.put(nm[0], nm[1], new ColumnInfo(internalName, valueInfo
- .getType(), nm[0], valueInfo.getIsVirtualCol(), valueInfo.isHiddenVirtualCol()));
- colExprMap.put(internalName, colDesc);
+ 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;
+ }
- SelectDesc select = new SelectDesc(exprs, outputs, false);
-
- SelectOperator sel = (SelectOperator) putOpInsertMap(
- OperatorFactory.getAndMakeChild(select, new RowSchema(inputRR
- .getColumnInfos()), input), inputRR, pGraphContext);
+ private ArrayList backtrack(List sources, Operator> current,
+ Operator> terminal, int tag) {
+ ArrayList result = new ArrayList();
+ for (ExprNodeDesc expr : sources) {
+ result.add(backtrack(expr, current, terminal, tag));
+ }
+ return result;
+ }
- sel.setColumnExprMap(colExprMap);
+ private ExprNodeDesc backtrack(ExprNodeDesc source, Operator> current,
+ Operator> terminal, int tag) {
+ if (source instanceof ExprNodeGenericFuncDesc) {
+ ExprNodeGenericFuncDesc function = (ExprNodeGenericFuncDesc) source.clone();
+ List params = new ArrayList();
+ for (ExprNodeDesc param : function.getChildren()) {
+ params.add(backtrack(param, current, terminal, tag));
+ }
+ function.setChildExprs(params);
+ return function;
+ }
+ if (source instanceof ExprNodeColumnDesc) {
+ ExprNodeColumnDesc column = (ExprNodeColumnDesc) source;
+ return backtrack(column, current, terminal, tag);
+ }
+ if (source instanceof ExprNodeFieldDesc) {
+ ExprNodeFieldDesc field = (ExprNodeFieldDesc) source;
+ String name = field.getFieldName();
+ TypeInfo type = field.getTypeInfo();
+ ExprNodeDesc backtrack = backtrack(field.getDesc(), current, terminal, tag);
+ return new ExprNodeFieldDesc(type, backtrack, name, field.getIsList());
+ }
+ return source;
+ }
- // Insert the select operator in between.
- sel.setChildOperators(childOp);
- for (Operator extends OperatorDesc> ch : childOp) {
- ch.replaceParent(oldParent, sel);
+ private ExprNodeDesc backtrack(ExprNodeColumnDesc column, Operator> current,
+ Operator> terminal, int tag) {
+ if (current == null || current == terminal) {
+ return column;
+ }
+ Map mapping = current.getColumnExprMap();
+ if (mapping == null || !mapping.containsKey(column.getColumn())) {
+ return backtrack(column, getSingleParent(current, tag), terminal, tag);
+ }
+ ExprNodeDesc mapped = mapping.get(column.getColumn());
+ if (mapped instanceof ExprNodeColumnDesc) {
+ String table1 = column.getTabAlias();
+ String table2 = ((ExprNodeColumnDesc)mapped).getTabAlias();
+ if (table1 != null && !table1.isEmpty() && table2 != null && !table2.isEmpty() &&
+ !table1.equals(table2)) {
+ return null;
}
+ }
+ return backtrack(mapped, getSingleParent(current, tag), terminal, tag);
+ }
+ protected Operator> getSingleParent(Operator> operator) {
+ if (operator.getParentOperators() != null && operator.getParentOperators().size() == 1) {
+ return operator.getParentOperators().get(0);
}
+ return null;
+ }
+
+ protected T getSingleParent(Operator> operator, Class type) {
+ Operator> parent = getSingleParent(operator);
+ return type.isInstance(parent) ? (T)parent : null;
+ }
- private Operator extends OperatorDesc> putOpInsertMap(
- Operator extends OperatorDesc> op, RowResolver rr, ParseContext pGraphContext) {
- OpParseContext ctx = new OpParseContext(rr);
- pGraphContext.getOpParseCtx().put(op, ctx);
- return op;
+ protected Operator> getSingleChild(Operator> operator) {
+ if (operator.getChildOperators() != null && operator.getChildOperators().size() == 1) {
+ return operator.getChildOperators().get(0);
}
+ return null;
+ }
- private boolean compareReduceSink(ReduceSinkOperator childReduceSink,
- ReduceSinkOperator parentRS,
- HashMap childColumnMapping,
- HashMap parentColumnMapping) {
+ protected Operator> getSingleParent(Operator> operator, int tag) {
+ return operator instanceof JoinOperator ? operator.getParentOperators().get(tag)
+ : getSingleParent(operator);
+ }
- ArrayList childPartitionCols = childReduceSink.getConf().getPartitionCols();
- ArrayList parentPartitionCols = parentRS.getConf().getPartitionCols();
+ protected > T findPossibleParent(Operator> start, Class target,
+ boolean trustScript) {
+ T[] parents = findPossibleParents(start, target, trustScript);
+ return parents != null && parents.length == 1 ? parents[0] : null;
+ }
- boolean ret = compareExprNodes(childColumnMapping, parentColumnMapping,
- childPartitionCols, parentPartitionCols);
- if (!ret) {
- return false;
+ @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;
}
-
- ArrayList childReduceKeyCols = childReduceSink.getConf().getKeyCols();
- ArrayList parentReduceKeyCols = parentRS.getConf().getKeyCols();
- ret = compareExprNodes(childColumnMapping, parentColumnMapping,
- childReduceKeyCols, parentReduceKeyCols);
- if (!ret) {
- return false;
+ if (cursor instanceof JoinOperator) {
+ return findParents((JoinOperator) cursor, target);
}
-
- String childRSOrder = childReduceSink.getConf().getOrder();
- String parentRSOrder = parentRS.getConf().getOrder();
- boolean moveChildRSOrderToParent = false;
- //move child reduce sink's order to the parent reduce sink operator.
- if (childRSOrder != null && !(childRSOrder.trim().equals(""))) {
- if (parentRSOrder == null
- || !childRSOrder.trim().equals(parentRSOrder.trim())) {
- return false;
- }
- } else {
- if(parentRSOrder == null || parentRSOrder.trim().equals("")) {
- moveChildRSOrderToParent = true;
- }
+ 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;
+ }
- int childNumReducers = childReduceSink.getConf().getNumReducers();
- int parentNumReducers = parentRS.getConf().getNumReducers();
- boolean moveChildReducerNumToParent = false;
- //move child reduce sink's number reducers to the parent reduce sink operator.
- if (childNumReducers != parentNumReducers) {
- if (childNumReducers == -1) {
- //do nothing.
- } else if (parentNumReducers == -1) {
- //set childNumReducers in the parent reduce sink operator.
- moveChildReducerNumToParent = true;
- } else {
- return false;
+ @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(moveChildRSOrderToParent) {
- parentRS.getConf().setOrder(childRSOrder);
+ if (result[tag] == null) {
+ throw new IllegalStateException("failed to find " + target.getSimpleName()
+ + " from " + join + " on tag " + tag);
}
+ }
+ return result;
+ }
- if(moveChildReducerNumToParent) {
- parentRS.getConf().setNumReducers(childNumReducers);
- }
+ 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;
+ }
- return true;
+ 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;
+ }
- private boolean compareExprNodes(HashMap childColumnMapping,
- HashMap parentColumnMapping,
- ArrayList childColExprs,
- ArrayList parentColExprs) {
+ protected void removeReduceSinkForGroupBy(ReduceSinkOperator cRS, GroupByOperator cGBYr,
+ ParseContext context) throws SemanticException {
- boolean childEmpty = childColExprs == null || childColExprs.size() == 0;
- boolean parentEmpty = parentColExprs == null || parentColExprs.size() == 0;
+ Operator> parent = getSingleParent(cRS);
- if (childEmpty) { //both empty
- return true;
- }
+ if (parent instanceof GroupByOperator) {
+ GroupByOperator cGBYm = (GroupByOperator) parent;
- //child not empty here
- if (parentEmpty) { // child not empty, but parent empty
- return false;
+ cGBYr.getConf().setKeys(cGBYm.getConf().getKeys());
+ cGBYr.getConf().setAggregators(cGBYm.getConf().getAggregators());
+ for (AggregationDesc aggr : cGBYm.getConf().getAggregators()) {
+ aggr.setMode(GenericUDAFEvaluator.Mode.COMPLETE);
}
-
- if (childColExprs.size() != parentColExprs.size()) {
- return false;
+ cGBYr.setColumnExprMap(cGBYm.getColumnExprMap());
+ cGBYr.setSchema(cGBYm.getSchema());
+ RowResolver resolver = context.getOpParseCtx().get(cGBYm).getRowResolver();
+ context.getOpParseCtx().get(cGBYr).setRowResolver(resolver);
+ } else {
+ cGBYr.getConf().setKeys(backtrack(cGBYr.getConf().getKeys(), cGBYr, parent, -1));
+ for (AggregationDesc aggr : cGBYr.getConf().getAggregators()) {
+ aggr.setParameters(backtrack(aggr.getParameters(), cGBYr, parent, -1));
}
- int i = 0;
- while (i < childColExprs.size()) {
- ExprNodeDesc childExpr = childColExprs.get(i);
- ExprNodeDesc parentExpr = parentColExprs.get(i);
-
- if ((childExpr instanceof ExprNodeColumnDesc)
- && (parentExpr instanceof ExprNodeColumnDesc)) {
- String childCol = childColumnMapping
- .get(((ExprNodeColumnDesc) childExpr).getColumn());
- String parentCol = parentColumnMapping
- .get(((ExprNodeColumnDesc) childExpr).getColumn());
-
- if (!childCol.equals(parentCol)) {
- return false;
- }
- } else {
- return false;
+
+ 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 = backtrack(oldMap.get(colName), cGBYr, parent, 0);
+ if (colExpr != null) {
+ newMap.put(colInfo.getInternalName(), colExpr);
}
- i++;
}
- return true;
+ cGBYr.setColumnExprMap(newMap);
+ cGBYr.setSchema(new RowSchema(newRR.getColumnInfos()));
+ context.getOpParseCtx().get(cGBYr).setRowResolver(newRR);
}
+ cGBYr.getConf().setMode(GroupByDesc.Mode.COMPLETE);
- /*
- * back track column names to find their corresponding original column
- * names. Only allow simple operators like 'select column' or filter.
- */
- private boolean backTrackColumnNames(
- HashMap columnMapping,
- ReduceSinkOperator reduceSink,
- Operator extends OperatorDesc> stopBacktrackFlagOp,
- ParseContext pGraphContext) {
- Operator extends OperatorDesc> startOperator = reduceSink;
- while (startOperator != null && startOperator != stopBacktrackFlagOp) {
- startOperator = startOperator.getParentOperators().get(0);
- Map colExprMap = startOperator.getColumnExprMap();
- if(colExprMap == null || colExprMap.size()==0) {
- continue;
- }
- Iterator keyIter = columnMapping.keySet().iterator();
- while (keyIter.hasNext()) {
- String key = keyIter.next();
- String oldCol = columnMapping.get(key);
- ExprNodeDesc exprNode = colExprMap.get(oldCol);
- if(exprNode instanceof ExprNodeColumnDesc) {
- String col = ((ExprNodeColumnDesc)exprNode).getColumn();
- columnMapping.put(key, col);
- } else {
- return false;
- }
- }
- }
+ 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 AbsctractReducerReducerProc {
+
+ // pRS-pGBY-cRS
+ @Override
+ 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;
+ }
- private HashMap getPartitionAndKeyColumnMapping(ReduceSinkOperator reduceSink) {
- HashMap columnMapping = new HashMap ();
- ReduceSinkDesc reduceSinkDesc = reduceSink.getConf();
- ArrayList partitionCols = reduceSinkDesc.getPartitionCols();
- ArrayList reduceKeyCols = reduceSinkDesc.getKeyCols();
- if(partitionCols != null) {
- for (ExprNodeDesc desc : partitionCols) {
- List cols = desc.getCols();
- for(String col : cols) {
- columnMapping.put(col, col);
- }
- }
- }
- if(reduceKeyCols != null) {
- for (ExprNodeDesc desc : reduceKeyCols) {
- List cols = desc.getCols();
- for(String col : cols) {
- columnMapping.put(col, col);
- }
- }
- }
- return columnMapping;
+ // pRS-pGBY-cRS-cGBY
+ @Override
+ 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;
+ }
+ }
- private ReduceSinkOperator findSingleParentReduceSink(ReduceSinkOperator childReduceSink, ParseContext pGraphContext) {
- Operator extends OperatorDesc> start = childReduceSink;
- while(start != null) {
- if (start.getParentOperators() == null
- || start.getParentOperators().size() != 1) {
- // this potentially is a join operator
- return null;
- }
+ static class JoinReducerProc extends AbsctractReducerReducerProc {
- boolean allowed = false;
- if ((start instanceof SelectOperator)
- || (start instanceof FilterOperator)
- || (start instanceof ExtractOperator)
- || (start instanceof ForwardOperator)
- || (start instanceof ScriptOperator)
- || (start instanceof ReduceSinkOperator)) {
- allowed = true;
- }
+ // pRS-pJOIN-cRS
+ @Override
+ 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;
+ }
- if (!allowed) {
- return null;
- }
+ // pRS-pJOIN-cRS-cGBY
+ @Override
+ 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;
+ }
+ }
- if ((start instanceof ScriptOperator)
- && !HiveConf.getBoolVar(pGraphContext.getConf(),
- HiveConf.ConfVars.HIVESCRIPTOPERATORTRUST)) {
- return null;
- }
+ static class ReducerReducerProc extends AbsctractReducerReducerProc {
- start = start.getParentOperators().get(0);
- if(start instanceof ReduceSinkOperator) {
- return (ReduceSinkOperator)start;
- }
- }
- return null;
+ // pRS-cRS
+ @Override
+ 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
+ @Override
+ 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;
+ }
}
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java
index ce67b33..e712223 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java
@@ -257,7 +257,7 @@ private void mergeMapJoinTaskWithChildMapJoinTask(MapRedTask task) {
// whether it contains common join op; if contains, return this common join op
JoinOperator joinOp = getJoinOp(currTask);
- if (joinOp == null) {
+ if (joinOp == null || joinOp.getConf().isFixedAsSorted()) {
return null;
}
currTask.setTaskTag(Task.COMMON_JOIN);
diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinProcFactory.java
index 7cff3d7..58e373e 100644
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinProcFactory.java
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SkewJoinProcFactory.java
@@ -52,6 +52,9 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx ctx,
Object... nodeOutputs) throws SemanticException {
SkewJoinProcCtx context = (SkewJoinProcCtx) ctx;
JoinOperator op = (JoinOperator) nd;
+ if (op.getConf().isFixedAsSorted()) {
+ return null;
+ }
ParseContext parseContext = context.getParseCtx();
Task extends Serializable> currentTsk = context.getCurrentTask();
GenMRSkewJoinProcessor.processSkewJoin(op, currentTsk, parseContext);
diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
index 3b1aa85..77abb43 100644
--- ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
+++ ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
@@ -80,6 +80,10 @@
protected Byte[] tagOrder;
private TableDesc keyTableDesc;
+ // this operator cannot be converted to mapjoin cause output is expected to be sorted on join key
+ // it's resulted from RS-dedup optimization, which removes following RS under some condition
+ private boolean fixedAsSorted;
+
public JoinDesc() {
}
@@ -507,4 +511,12 @@ public void setFilterMap(int[][] filterMap) {
}
return null;
}
+
+ public boolean isFixedAsSorted() {
+ return fixedAsSorted;
+ }
+
+ public void setFixedAsSorted(boolean fixedAsSorted) {
+ this.fixedAsSorted = fixedAsSorted;
+ }
}
diff --git ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
index 9a0968a..ee346bd 100644
--- ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
+++ ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
@@ -133,7 +133,7 @@ public TransitiveContext() {
newFilters = new HashMap();
}
- public Map getFilterPropates() {
+ public Map getFilterPropagates() {
return filterPropagates;
}
@@ -153,7 +153,7 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
int srcPos = join.getParentOperators().indexOf(source);
TransitiveContext context = (TransitiveContext) procCtx;
- Map filterPropagates = context.getFilterPropates();
+ Map filterPropagates = context.getFilterPropagates();
Map newFilters = context.getNewfilters();
int[][] targets = filterPropagates.get(join);
diff --git ql/src/test/queries/clientpositive/groupby_distinct_samekey.q ql/src/test/queries/clientpositive/groupby_distinct_samekey.q
index da29b9d..8219a5e 100644
--- ql/src/test/queries/clientpositive/groupby_distinct_samekey.q
+++ ql/src/test/queries/clientpositive/groupby_distinct_samekey.q
@@ -1,7 +1,10 @@
--- This test covers HIVE-2322
+-- This test covers HIVE-2332
create table t1 (int1 int, int2 int, str1 string, str2 string);
+set hive.optimize.reducededuplication=false;
+--disabled RS-dedup for keeping intention of test
+
insert into table t1 select cast(key as int), cast(key as int), value, value from src where key < 6;
explain select Q1.int1, sum(distinct Q1.int1) from (select * from t1 order by int1) Q1 group by Q1.int1;
explain select int1, sum(distinct int1) from t1 group by int1;
diff --git ql/src/test/queries/clientpositive/join31.q ql/src/test/queries/clientpositive/join31.q
index 0ba143a..d731471 100644
--- ql/src/test/queries/clientpositive/join31.q
+++ ql/src/test/queries/clientpositive/join31.q
@@ -4,6 +4,7 @@ set hive.auto.convert.join=true;
set hive.auto.convert.join.noconditionaltask=true;
set hive.auto.convert.join.noconditionaltask.size=10000;
+set hive.optimize.reducededuplication=false;
-- Since the inputs are small, it should be automatically converted to mapjoin
EXPLAIN
@@ -20,3 +21,20 @@ FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN
group by subq1.key;
select * from dest_j1 x order by x.key;
+
+set hive.optimize.reducededuplication=true;
+
+EXPLAIN
+INSERT OVERWRITE TABLE dest_j1
+SELECT subq1.key, count(1) as cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN
+ (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key)
+group by subq1.key;
+
+INSERT OVERWRITE TABLE dest_j1
+SELECT subq1.key, count(1) as cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN
+ (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key)
+group by subq1.key;
+
+select * from dest_j1 x order by x.key;
diff --git ql/src/test/queries/clientpositive/reduce_deduplicate.q ql/src/test/queries/clientpositive/reduce_deduplicate.q
index 0d25754..13b4060 100644
--- ql/src/test/queries/clientpositive/reduce_deduplicate.q
+++ ql/src/test/queries/clientpositive/reduce_deduplicate.q
@@ -3,9 +3,11 @@ set hive.enforce.bucketing = true;
set hive.exec.reducers.max = 1;
set hive.exec.script.trust = true;
-
+set hive.optimize.reducededuplication=true;
+set hive.optimize.reducededuplication.min.reducer=1;
CREATE TABLE bucket5_1(key string, value string) CLUSTERED BY (key) INTO 2 BUCKETS;
+
explain extended
insert overwrite table bucket5_1
select * from src cluster by key;
@@ -13,7 +15,17 @@ select * from src cluster by key;
insert overwrite table bucket5_1
select * from src cluster by key;
+set hive.optimize.reducededuplication.min.reducer=4;
+
+-- not RRed cause of number of reducer (1) is lesser than default of hive.optimize.reducededuplication.min.reducer (4)
+explain select sum(hash(key)),sum(hash(value)) from bucket5_1;
+
+set hive.optimize.reducededuplication.min.reducer=1;
+
+explain select sum(hash(key)),sum(hash(value)) from bucket5_1;
+
select sum(hash(key)),sum(hash(value)) from bucket5_1;
+
select sum(hash(key)),sum(hash(value)) from src;
@@ -22,6 +34,8 @@ create table complex_tbl_1(aid string, bid string, t int, ctime string, etime bi
create table complex_tbl_2(aet string, aes string) partitioned by (ds string);
+set hive.optimize.reducededuplication.min.reducer=4;
+
explain extended
insert overwrite table complex_tbl_1 partition (ds='2010-03-29')
select s2.* from
@@ -38,7 +52,20 @@ select s2.* from
)s
)s2;
+set hive.optimize.reducededuplication.min.reducer=1;
-
-
-
+explain extended
+insert overwrite table complex_tbl_1 partition (ds='2010-03-29')
+select s2.* from
+(
+ select TRANSFORM (aid,bid,t,ctime,etime,l,et)
+ USING 'cat'
+ AS (aid string, bid string, t int, ctime string, etime bigint, l string, et string)
+ from
+ (
+ select transform(aet,aes)
+ using 'cat'
+ as (aid string, bid string, t int, ctime string, etime bigint, l string, et string)
+ from complex_tbl_2 where ds ='2010-03-29' cluster by bid
+)s
+)s2;
diff --git ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q
new file mode 100644
index 0000000..09cc1c2
--- /dev/null
+++ ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q
@@ -0,0 +1,36 @@
+set hive.optimize.reducededuplication=true;
+set hive.map.aggr=true;
+
+explain select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key;
+explain select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value);
+explain select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1);
+explain select key, sum(key) as value from src group by key order by key, value;
+explain select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value;
+explain select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value;
+explain from (select key, value from src group by key, value) s select s.key group by s.key;
+
+select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key;
+select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value);
+select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1);
+select key, sum(key) as value from src group by key order by key, value;
+select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value;
+select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value;
+from (select key, value from src group by key, value) s select s.key group by s.key;
+
+set hive.map.aggr=false;
+
+explain select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key;
+explain select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value);
+explain select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1);
+explain select key, sum(key) as value from src group by key order by key, value;
+explain select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value;
+explain select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value;
+explain from (select key, value from src group by key, value) s select s.key group by s.key;
+
+select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key;
+select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value);
+select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1);
+select key, sum(key) as value from src group by key order by key, value;
+select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value;
+select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value;
+from (select key, value from src group by key, value) s select s.key group by s.key;
diff --git ql/src/test/results/clientpositive/cluster.q.out ql/src/test/results/clientpositive/cluster.q.out
index 353cc70..16c94ba 100644
--- ql/src/test/results/clientpositive/cluster.q.out
+++ ql/src/test/results/clientpositive/cluster.q.out
@@ -818,7 +818,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -887,27 +886,8 @@ STAGE PLANS:
expr: _col5
type: string
outputColumnNames: _col0, _col1, _col2, _col3
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
+ Select Operator
+ expressions:
expr: _col0
type: string
expr: _col1
@@ -916,14 +896,13 @@ STAGE PLANS:
type: string
expr: _col3
type: string
- Reduce Operator Tree:
- Extract
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ outputColumnNames: _col0, _col1, _col2, _col3
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
@@ -950,7 +929,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -1015,41 +993,21 @@ STAGE PLANS:
expr: _col4
type: string
outputColumnNames: _col0, _col1, _col2
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
+ Select Operator
+ expressions:
expr: _col0
type: string
expr: _col1
type: string
expr: _col2
type: string
- Reduce Operator Tree:
- Extract
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
diff --git ql/src/test/results/clientpositive/groupby2.q.out ql/src/test/results/clientpositive/groupby2.q.out
index 7047d46..1eec1d8 100644
--- ql/src/test/results/clientpositive/groupby2.q.out
+++ ql/src/test/results/clientpositive/groupby2.q.out
@@ -16,9 +16,8 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
STAGE PLANS:
Stage: Stage-1
@@ -56,43 +55,7 @@ STAGE PLANS:
keys:
expr: KEY._col0
type: string
- mode: partial1
- outputColumnNames: _col0, _col1, _col2
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col1
- type: bigint
- expr: _col2
- type: double
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: count(VALUE._col0)
- expr: sum(VALUE._col1)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- mode: final
+ mode: complete
outputColumnNames: _col0, _col1, _col2
Select Operator
expressions:
@@ -131,7 +94,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: default.dest_g2
- Stage: Stage-3
+ Stage: Stage-2
Stats-Aggr Operator
diff --git ql/src/test/results/clientpositive/groupby2_map_skew.q.out ql/src/test/results/clientpositive/groupby2_map_skew.q.out
index 608ae0c..76eede3 100644
--- ql/src/test/results/clientpositive/groupby2_map_skew.q.out
+++ ql/src/test/results/clientpositive/groupby2_map_skew.q.out
@@ -16,9 +16,8 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
- Stage-3 depends on stages: Stage-0
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
STAGE PLANS:
Stage: Stage-1
@@ -73,43 +72,7 @@ STAGE PLANS:
keys:
expr: KEY._col0
type: string
- mode: partials
- outputColumnNames: _col0, _col1, _col2
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col1
- type: bigint
- expr: _col2
- type: double
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: count(VALUE._col0)
- expr: sum(VALUE._col1)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- mode: final
+ mode: complete
outputColumnNames: _col0, _col1, _col2
Select Operator
expressions:
@@ -148,7 +111,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: default.dest1
- Stage: Stage-3
+ Stage: Stage-2
Stats-Aggr Operator
diff --git ql/src/test/results/clientpositive/groupby_cube1.q.out ql/src/test/results/clientpositive/groupby_cube1.q.out
index 8cc2471..974230d 100644
--- ql/src/test/results/clientpositive/groupby_cube1.q.out
+++ ql/src/test/results/clientpositive/groupby_cube1.q.out
@@ -411,7 +411,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -471,44 +470,7 @@ STAGE PLANS:
type: string
expr: KEY._col1
type: string
- mode: partials
- outputColumnNames: _col0, _col1, _col2
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- expr: _col1
- type: string
- sort order: ++
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col2
- type: bigint
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: count(VALUE._col0)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- expr: KEY._col1
- type: string
- mode: final
+ mode: complete
outputColumnNames: _col0, _col1, _col2
Select Operator
expressions:
diff --git ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out
index 26566e8..aa4696c 100644
--- ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out
+++ ql/src/test/results/clientpositive/groupby_distinct_samekey.q.out
@@ -1,17 +1,21 @@
-PREHOOK: query: -- This test covers HIVE-2322
+PREHOOK: query: -- This test covers HIVE-2332
create table t1 (int1 int, int2 int, str1 string, str2 string)
PREHOOK: type: CREATETABLE
-POSTHOOK: query: -- This test covers HIVE-2322
+POSTHOOK: query: -- This test covers HIVE-2332
create table t1 (int1 int, int2 int, str1 string, str2 string)
POSTHOOK: type: CREATETABLE
POSTHOOK: Output: default@t1
-PREHOOK: query: insert into table t1 select cast(key as int), cast(key as int), value, value from src where key < 6
+PREHOOK: query: --disabled RS-dedup for keeping intention of test
+
+insert into table t1 select cast(key as int), cast(key as int), value, value from src where key < 6
PREHOOK: type: QUERY
PREHOOK: Input: default@src
PREHOOK: Output: default@t1
-POSTHOOK: query: insert into table t1 select cast(key as int), cast(key as int), value, value from src where key < 6
+POSTHOOK: query: --disabled RS-dedup for keeping intention of test
+
+insert into table t1 select cast(key as int), cast(key as int), value, value from src where key < 6
POSTHOOK: type: QUERY
POSTHOOK: Input: default@src
POSTHOOK: Output: default@t1
diff --git ql/src/test/results/clientpositive/groupby_rollup1.q.out ql/src/test/results/clientpositive/groupby_rollup1.q.out
index bf4e0e4..0e04ec4 100644
--- ql/src/test/results/clientpositive/groupby_rollup1.q.out
+++ ql/src/test/results/clientpositive/groupby_rollup1.q.out
@@ -399,7 +399,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -459,44 +458,7 @@ STAGE PLANS:
type: string
expr: KEY._col1
type: string
- mode: partials
- outputColumnNames: _col0, _col1, _col2
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- expr: _col1
- type: string
- sort order: ++
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col2
- type: bigint
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: count(VALUE._col0)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- expr: KEY._col1
- type: string
- mode: final
+ mode: complete
outputColumnNames: _col0, _col1, _col2
Select Operator
expressions:
diff --git ql/src/test/results/clientpositive/index_bitmap3.q.out ql/src/test/results/clientpositive/index_bitmap3.q.out
index ac302ce..992b864 100644
--- ql/src/test/results/clientpositive/index_bitmap3.q.out
+++ ql/src/test/results/clientpositive/index_bitmap3.q.out
@@ -137,7 +137,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -239,54 +238,21 @@ STAGE PLANS:
keys:
expr: _col0
type: string
- mode: hash
+ mode: complete
outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col1
- type: array
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: collect_set(VALUE._col0)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- mode: mergepartial
- outputColumnNames: _col0, _col1
- Select Operator
- expressions:
- expr: _col0
- type: string
- expr: _col1
- type: array
- outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: array
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
diff --git ql/src/test/results/clientpositive/index_bitmap_auto.q.out ql/src/test/results/clientpositive/index_bitmap_auto.q.out
index 5526cb8..8ade13b 100644
--- ql/src/test/results/clientpositive/index_bitmap_auto.q.out
+++ ql/src/test/results/clientpositive/index_bitmap_auto.q.out
@@ -156,7 +156,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -258,54 +257,21 @@ STAGE PLANS:
keys:
expr: _col0
type: string
- mode: hash
+ mode: complete
outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col1
- type: array
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: collect_set(VALUE._col0)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- mode: mergepartial
- outputColumnNames: _col0, _col1
- Select Operator
- expressions:
- expr: _col0
- type: string
- expr: _col1
- type: array
- outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: array
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
diff --git ql/src/test/results/clientpositive/infer_bucket_sort.q.out ql/src/test/results/clientpositive/infer_bucket_sort.q.out
index 61d7ac2..0a27b8a 100644
--- ql/src/test/results/clientpositive/infer_bucket_sort.q.out
+++ ql/src/test/results/clientpositive/infer_bucket_sort.q.out
@@ -2032,11 +2032,11 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSche
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)c.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ]
@@ -2080,11 +2080,11 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSche
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)c.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ]
@@ -2122,7 +2122,7 @@ OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Compressed: No
Num Buckets: 1
Bucket Columns: [key]
-Sort Columns: []
+Sort Columns: [Order(col:key, order:1)]
Storage Desc Params:
serialization.format 1
PREHOOK: query: -- Test group by in subquery followed by sort by, should only be sorted by the sort key
@@ -2143,8 +2143,6 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE []
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
@@ -2172,11 +2170,13 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSche
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)c.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ]
@@ -2193,8 +2193,6 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE []
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
@@ -2222,11 +2220,13 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSche
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)c.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ]
@@ -2262,8 +2262,8 @@ SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Compressed: No
-Num Buckets: -1
-Bucket Columns: []
+Num Buckets: 1
+Bucket Columns: [key]
Sort Columns: [Order(col:key, order:1)]
Storage Desc Params:
serialization.format 1
@@ -2285,8 +2285,6 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE []
POSTHOOK: Lineage: test_table PARTITION(part=1).key SCRIPT [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.null, ]
@@ -2316,11 +2314,13 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSche
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)c.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ]
@@ -2337,8 +2337,6 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE []
POSTHOOK: Lineage: test_table PARTITION(part=1).key SCRIPT [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.null, ]
@@ -2368,11 +2366,13 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSche
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)c.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ]
@@ -2431,8 +2431,6 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE []
POSTHOOK: Lineage: test_table PARTITION(part=1).key SCRIPT [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.null, ]
@@ -2464,11 +2462,13 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSche
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)c.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ]
@@ -2485,8 +2485,6 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE []
POSTHOOK: Lineage: test_table PARTITION(part=1).key SCRIPT [(src)src.FieldSchema(name:key, type:string, comment:default), (src)src.null, ]
@@ -2518,11 +2516,13 @@ POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSche
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: test_table PARTITION(part=1).value EXPRESSION [(src)src.null, ]
+POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key SIMPLE [(src)a.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).value SIMPLE [(src)c.FieldSchema(name:value, type:string, comment:default), ]
POSTHOOK: Lineage: test_table PARTITION(part=1).key EXPRESSION [(src)src.null, ]
diff --git ql/src/test/results/clientpositive/join31.q.out ql/src/test/results/clientpositive/join31.q.out
index 763f2c5..5437843 100644
--- ql/src/test/results/clientpositive/join31.q.out
+++ ql/src/test/results/clientpositive/join31.q.out
@@ -427,3 +427,429 @@ POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string
406 1
66 1
98 1
+PREHOOK: query: EXPLAIN
+INSERT OVERWRITE TABLE dest_j1
+SELECT subq1.key, count(1) as cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN
+ (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key)
+group by subq1.key
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+INSERT OVERWRITE TABLE dest_j1
+SELECT subq1.key, count(1) as cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN
+ (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key)
+group by subq1.key
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: dest_j1.cnt EXPRESSION [(src1)x.null, (src)y.null, ]
+POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src1) x)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL x) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL x) key)))) subq1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src) y)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL y) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL y) key)))) subq2) (= (. (TOK_TABLE_OR_COL subq1) key) (. (TOK_TABLE_OR_COL subq2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL subq1) key)) (TOK_SELEXPR (TOK_FUNCTION count 1) cnt)) (TOK_GROUPBY (. (TOK_TABLE_OR_COL subq1) key))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-8 depends on stages: Stage-1, Stage-5 , consists of Stage-9, Stage-10, Stage-2
+ Stage-9 has a backup stage: Stage-2
+ Stage-6 depends on stages: Stage-9
+ Stage-3 depends on stages: Stage-2, Stage-6, Stage-7
+ Stage-0 depends on stages: Stage-3
+ Stage-4 depends on stages: Stage-0
+ Stage-10 has a backup stage: Stage-2
+ Stage-7 depends on stages: Stage-10
+ Stage-2
+ Stage-5 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ subq2:y
+ TableScan
+ alias: y
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-8
+ Conditional Operator
+
+ Stage: Stage-9
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ $INTNAME
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ $INTNAME
+ HashTable Sink Operator
+ condition expressions:
+ 0 {_col0}
+ 1
+ handleSkewJoin: false
+ keys:
+ 0 [Column[_col0]]
+ 1 [Column[_col0]]
+ Position of Big Table: 0
+
+ Stage: Stage-6
+ Map Reduce
+ Alias -> Map Operator Tree:
+ $INTNAME1
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0}
+ 1
+ handleSkewJoin: false
+ keys:
+ 0 [Column[_col0]]
+ 1 [Column[_col0]]
+ outputColumnNames: _col0
+ Position of Big Table: 0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ Local Work:
+ Map Reduce Local Work
+
+ Stage: Stage-3
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToInteger(_col1)
+ type: int
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.dest_j1
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ replace: true
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.dest_j1
+
+ Stage: Stage-4
+ Stats-Aggr Operator
+
+ Stage: Stage-10
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ $INTNAME1
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ $INTNAME1
+ HashTable Sink Operator
+ condition expressions:
+ 0 {_col0}
+ 1
+ handleSkewJoin: false
+ keys:
+ 0 [Column[_col0]]
+ 1 [Column[_col0]]
+ Position of Big Table: 1
+
+ Stage: Stage-7
+ Map Reduce
+ Alias -> Map Operator Tree:
+ $INTNAME
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0}
+ 1
+ handleSkewJoin: false
+ keys:
+ 0 [Column[_col0]]
+ 1 [Column[_col0]]
+ outputColumnNames: _col0
+ Position of Big Table: 1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ Local Work:
+ Map Reduce Local Work
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+ $INTNAME
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: 1
+ $INTNAME1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: 0
+ value expressions:
+ expr: _col0
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1
+ handleSkewJoin: false
+ outputColumnNames: _col0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-5
+ Map Reduce
+ Alias -> Map Operator Tree:
+ subq1:x
+ TableScan
+ alias: x
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+
+PREHOOK: query: INSERT OVERWRITE TABLE dest_j1
+SELECT subq1.key, count(1) as cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN
+ (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key)
+group by subq1.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+PREHOOK: Output: default@dest_j1
+POSTHOOK: query: INSERT OVERWRITE TABLE dest_j1
+SELECT subq1.key, count(1) as cnt
+FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN
+ (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key)
+group by subq1.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+POSTHOOK: Output: default@dest_j1
+POSTHOOK: Lineage: dest_j1.cnt EXPRESSION [(src1)x.null, (src)y.null, ]
+POSTHOOK: Lineage: dest_j1.cnt EXPRESSION [(src1)x.null, (src)y.null, ]
+POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
+PREHOOK: query: select * from dest_j1 x order by x.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dest_j1
+#### A masked pattern was here ####
+POSTHOOK: query: select * from dest_j1 x order by x.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dest_j1
+#### A masked pattern was here ####
+POSTHOOK: Lineage: dest_j1.cnt EXPRESSION [(src1)x.null, (src)y.null, ]
+POSTHOOK: Lineage: dest_j1.cnt EXPRESSION [(src1)x.null, (src)y.null, ]
+POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.key SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
+128 1
+146 1
+150 1
+213 1
+224 1
+238 1
+255 1
+273 1
+278 1
+311 1
+369 1
+401 1
+406 1
+66 1
+98 1
diff --git ql/src/test/results/clientpositive/ppd2.q.out ql/src/test/results/clientpositive/ppd2.q.out
index 65ba807..6856945 100644
--- ql/src/test/results/clientpositive/ppd2.q.out
+++ ql/src/test/results/clientpositive/ppd2.q.out
@@ -31,7 +31,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -61,7 +60,9 @@ STAGE PLANS:
key expressions:
expr: _col0
type: string
- sort order: +
+ expr: _col1
+ type: bigint
+ sort order: +-
Map-reduce partition columns:
expr: _col0
type: string
@@ -93,52 +94,30 @@ STAGE PLANS:
expr: _col1
type: bigint
outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- sort order: +-
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- Reduce Operator Tree:
- Extract
- Filter Operator
- predicate:
- expr: (_col1 > 1)
- type: boolean
- Select Operator
- expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Filter Operator
+ predicate:
+ expr: (_col1 > 1)
+ type: boolean
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
@@ -529,7 +508,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -563,7 +541,9 @@ STAGE PLANS:
key expressions:
expr: _col0
type: string
- sort order: +
+ expr: _col1
+ type: bigint
+ sort order: +-
Map-reduce partition columns:
expr: _col0
type: string
@@ -595,52 +575,30 @@ STAGE PLANS:
expr: _col1
type: bigint
outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- sort order: +-
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- Reduce Operator Tree:
- Extract
- Filter Operator
- predicate:
- expr: (_col1 > 1)
- type: boolean
- Select Operator
- expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Filter Operator
+ predicate:
+ expr: (_col1 > 1)
+ type: boolean
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
diff --git ql/src/test/results/clientpositive/ppd_gby_join.q.out ql/src/test/results/clientpositive/ppd_gby_join.q.out
index eb21387..c4352c4 100644
--- ql/src/test/results/clientpositive/ppd_gby_join.q.out
+++ ql/src/test/results/clientpositive/ppd_gby_join.q.out
@@ -23,7 +23,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -123,54 +122,21 @@ STAGE PLANS:
keys:
expr: _col0
type: string
- mode: hash
+ mode: complete
outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col1
- type: bigint
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: count(VALUE._col0)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- mode: mergepartial
- outputColumnNames: _col0, _col1
- Select Operator
- expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
@@ -202,7 +168,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -286,54 +251,21 @@ STAGE PLANS:
keys:
expr: _col0
type: string
- mode: hash
+ mode: complete
outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col1
- type: bigint
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: count(VALUE._col0)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- mode: mergepartial
- outputColumnNames: _col0, _col1
- Select Operator
- expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
diff --git ql/src/test/results/clientpositive/reduce_deduplicate.q.out ql/src/test/results/clientpositive/reduce_deduplicate.q.out
index 28f592e..933c2b3 100644
--- ql/src/test/results/clientpositive/reduce_deduplicate.q.out
+++ ql/src/test/results/clientpositive/reduce_deduplicate.q.out
@@ -171,6 +171,146 @@ POSTHOOK: Input: default@src
POSTHOOK: Output: default@bucket5_1
POSTHOOK: Lineage: bucket5_1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: bucket5_1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: -- not RRed cause of number of reducer (1) is lesser than default of hive.optimize.reducededuplication.min.reducer (4)
+explain select sum(hash(key)),sum(hash(value)) from bucket5_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- not RRed cause of number of reducer (1) is lesser than default of hive.optimize.reducededuplication.min.reducer (4)
+explain select sum(hash(key)),sum(hash(value)) from bucket5_1
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: bucket5_1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: bucket5_1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME bucket5_1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (TOK_TABLE_OR_COL key)))) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (TOK_TABLE_OR_COL value)))))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ bucket5_1
+ TableScan
+ alias: bucket5_1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ outputColumnNames: key, value
+ Group By Operator
+ aggregations:
+ expr: sum(hash(key))
+ expr: sum(hash(value))
+ bucketGroup: false
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ sort order:
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: bigint
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: sum(VALUE._col0)
+ expr: sum(VALUE._col1)
+ bucketGroup: false
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: bigint
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select sum(hash(key)),sum(hash(value)) from bucket5_1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select sum(hash(key)),sum(hash(value)) from bucket5_1
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: bucket5_1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: bucket5_1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME bucket5_1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (TOK_TABLE_OR_COL key)))) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (TOK_TABLE_OR_COL value)))))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ bucket5_1
+ TableScan
+ alias: bucket5_1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ outputColumnNames: key, value
+ Group By Operator
+ aggregations:
+ expr: sum(hash(key))
+ expr: sum(hash(value))
+ bucketGroup: false
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ sort order:
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: bigint
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: sum(VALUE._col0)
+ expr: sum(VALUE._col1)
+ bucketGroup: false
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: bigint
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
PREHOOK: query: select sum(hash(key)),sum(hash(value)) from bucket5_1
PREHOOK: type: QUERY
PREHOOK: Input: default@bucket5_1
@@ -407,3 +547,203 @@ STAGE PLANS:
#### A masked pattern was here ####
+PREHOOK: query: explain extended
+insert overwrite table complex_tbl_1 partition (ds='2010-03-29')
+select s2.* from
+(
+ select TRANSFORM (aid,bid,t,ctime,etime,l,et)
+ USING 'cat'
+ AS (aid string, bid string, t int, ctime string, etime bigint, l string, et string)
+ from
+ (
+ select transform(aet,aes)
+ using 'cat'
+ as (aid string, bid string, t int, ctime string, etime bigint, l string, et string)
+ from complex_tbl_2 where ds ='2010-03-29' cluster by bid
+)s
+)s2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+insert overwrite table complex_tbl_1 partition (ds='2010-03-29')
+select s2.* from
+(
+ select TRANSFORM (aid,bid,t,ctime,etime,l,et)
+ USING 'cat'
+ AS (aid string, bid string, t int, ctime string, etime bigint, l string, et string)
+ from
+ (
+ select transform(aet,aes)
+ using 'cat'
+ as (aid string, bid string, t int, ctime string, etime bigint, l string, et string)
+ from complex_tbl_2 where ds ='2010-03-29' cluster by bid
+)s
+)s2
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: bucket5_1.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: bucket5_1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME complex_tbl_2))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TRANSFORM (TOK_EXPLIST (TOK_TABLE_OR_COL aet) (TOK_TABLE_OR_COL aes)) TOK_SERDE TOK_RECORDWRITER 'cat' TOK_SERDE TOK_RECORDREADER (TOK_TABCOLLIST (TOK_TABCOL aid TOK_STRING) (TOK_TABCOL bid TOK_STRING) (TOK_TABCOL t TOK_INT) (TOK_TABCOL ctime TOK_STRING) (TOK_TABCOL etime TOK_BIGINT) (TOK_TABCOL l TOK_STRING) (TOK_TABCOL et TOK_STRING))))) (TOK_WHERE (= (TOK_TABLE_OR_COL ds) '2010-03-29')) (TOK_CLUSTERBY (TOK_TABLE_OR_COL bid)))) s)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TRANSFORM (TOK_EXPLIST (TOK_TABLE_OR_COL aid) (TOK_TABLE_OR_COL bid) (TOK_TABLE_OR_COL t) (TOK_TABLE_OR_COL ctime) (TOK_TABLE_OR_COL etime) (TOK_TABLE_OR_COL l) (TOK_TABLE_OR_COL et)) TOK_SERDE TOK_RECORDWRITER 'cat' TOK_SERDE TOK_RECORDREADER (TOK_TABCOLLIST (TOK_TABCOL aid TOK_STRING) (TOK_TABCOL bid TOK_STRING) (TOK_TABCOL t TOK_INT) (TOK_TABCOL ctime TOK_STRING) (TOK_TABCOL etime TOK_BIGINT) (TOK_TABCOL l TOK_STRING) (TOK_TABCOL et TOK_STRING))))))) s2)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME complex_tbl_1) (TOK_PARTSPEC (TOK_PARTVAL ds '2010-03-29')))) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME s2))))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+ Stage-2 depends on stages: Stage-0
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ s2:s:complex_tbl_2
+ TableScan
+ alias: complex_tbl_2
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate:
+ expr: (ds = '2010-03-29')
+ type: boolean
+ Select Operator
+ expressions:
+ expr: aet
+ type: string
+ expr: aes
+ type: string
+ outputColumnNames: _col0, _col1
+ Transform Operator
+ command: cat
+ output info:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ columns _col0,_col1,_col2,_col3,_col4,_col5,_col6
+ columns.types string,string,int,string,bigint,string,string
+ field.delim 9
+ serialization.format 9
+ Reduce Output Operator
+ key expressions:
+ expr: _col1
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col1
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col2
+ type: int
+ expr: _col3
+ type: string
+ expr: _col4
+ type: bigint
+ expr: _col5
+ type: string
+ expr: _col6
+ type: string
+ Needs Tagging: false
+ Reduce Operator Tree:
+ Extract
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col2
+ type: int
+ expr: _col3
+ type: string
+ expr: _col4
+ type: bigint
+ expr: _col5
+ type: string
+ expr: _col6
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ Transform Operator
+ command: cat
+ output info:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ columns _col0,_col1,_col2,_col3,_col4,_col5,_col6
+ columns.types string,string,int,string,bigint,string,string
+ field.delim 9
+ serialization.format 9
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col2
+ type: int
+ expr: _col3
+ type: string
+ expr: _col4
+ type: bigint
+ expr: _col5
+ type: string
+ expr: _col6
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
+ Static Partition Specification: ds=2010-03-29/
+#### A masked pattern was here ####
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ bucket_count -1
+ columns aid,bid,t,ctime,etime,l,et
+ columns.types string:string:int:string:bigint:string:string
+#### A masked pattern was here ####
+ name default.complex_tbl_1
+ partition_columns ds
+ serialization.ddl struct complex_tbl_1 { string aid, string bid, i32 t, string ctime, i64 etime, string l, string et}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.complex_tbl_1
+ TotalFiles: 1
+ GatherStats: true
+ MultiFileSpray: false
+
+ Stage: Stage-0
+ Move Operator
+ tables:
+ partition:
+ ds 2010-03-29
+ replace: true
+#### A masked pattern was here ####
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ bucket_count -1
+ columns aid,bid,t,ctime,etime,l,et
+ columns.types string:string:int:string:bigint:string:string
+#### A masked pattern was here ####
+ name default.complex_tbl_1
+ partition_columns ds
+ serialization.ddl struct complex_tbl_1 { string aid, string bid, i32 t, string ctime, i64 etime, string l, string et}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.complex_tbl_1
+#### A masked pattern was here ####
+
+ Stage: Stage-2
+ Stats-Aggr Operator
+#### A masked pattern was here ####
+
+
diff --git ql/src/test/results/clientpositive/reduce_deduplicate_extended.q.out ql/src/test/results/clientpositive/reduce_deduplicate_extended.q.out
new file mode 100644
index 0000000..4cd7816
--- /dev/null
+++ ql/src/test/results/clientpositive/reduce_deduplicate_extended.q.out
@@ -0,0 +1,4607 @@
+PREHOOK: query: explain select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL value))))) Q1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL key)))) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ q1:src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ Reduce Operator Tree:
+ Extract
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Group By Operator
+ aggregations:
+ expr: sum(_col0)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ mode: complete
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) Q1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL key))) (TOK_SELEXPR (TOK_FUNCTION lower (TOK_TABLE_OR_COL value)))) (TOK_GROUPBY (TOK_TABLE_OR_COL key) (TOK_FUNCTION lower (TOK_TABLE_OR_COL value)))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ q1:src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: lower(_col1)
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: lower(_col1)
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ Reduce Operator Tree:
+ Extract
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Group By Operator
+ aggregations:
+ expr: sum(_col0)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ expr: lower(_col1)
+ type: string
+ mode: complete
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col2
+ type: double
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (+ (TOK_TABLE_OR_COL value) 1) X)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) Q1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL key))) (TOK_SELEXPR (+ (TOK_TABLE_OR_COL X) 1))) (TOK_GROUPBY (TOK_TABLE_OR_COL key) (+ (TOK_TABLE_OR_COL X) 1))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ q1:src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: (value + 1)
+ type: double
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: (_col1 + 1)
+ type: double
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: (_col1 + 1)
+ type: double
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ Reduce Operator Tree:
+ Extract
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ outputColumnNames: _col0, _col1
+ Group By Operator
+ aggregations:
+ expr: sum(_col0)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ expr: (_col1 + 1)
+ type: double
+ mode: complete
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col2
+ type: double
+ expr: _col1
+ type: double
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select key, sum(key) as value from src group by key order by key, value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select key, sum(key) as value from src group by key order by key, value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL key)) value)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL value)))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: sum(key)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: double
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: sum(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ sort order: ++
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ Reduce Operator Tree:
+ Extract
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src)) (TOK_TABREF (TOK_TABNAME src1)) (= (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL src1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key)) (TOK_SELEXPR (TOK_FUNCTION sum (. (TOK_TABLE_OR_COL src) key)))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL src) value))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ src
+ TableScan
+ alias: src
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ src1
+ TableScan
+ alias: src1
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Group By Operator
+ aggregations:
+ expr: sum(_col0)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col2
+ type: double
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: sum(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ expr: KEY._col1
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col2
+ type: double
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src)) (TOK_TABREF (TOK_TABNAME src1)) (= (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL src1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src) value)))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ src
+ TableScan
+ alias: src
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ src1
+ TableScan
+ alias: src1
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ Reduce Operator Tree:
+ Extract
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain from (select key, value from src group by key, value) s select s.key group by s.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain from (select key, value from src group by key, value) s select s.key group by s.key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_TABLE_OR_COL value))) (TOK_GROUPBY (TOK_TABLE_OR_COL key) (TOK_TABLE_OR_COL value)))) s)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL s) key))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL s) key))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ s:src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ outputColumnNames: key, value
+ Group By Operator
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ tag: -1
+ Reduce Operator Tree:
+ Group By Operator
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ expr: KEY._col1
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Group By Operator
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ mode: complete
+ outputColumnNames: _col0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0 0.0
+10 10.0
+100 200.0
+103 206.0
+104 208.0
+105 105.0
+11 11.0
+111 111.0
+113 226.0
+114 114.0
+116 116.0
+118 236.0
+119 357.0
+12 24.0
+120 240.0
+125 250.0
+126 126.0
+128 384.0
+129 258.0
+131 131.0
+133 133.0
+134 268.0
+136 136.0
+137 274.0
+138 552.0
+143 143.0
+145 145.0
+146 292.0
+149 298.0
+15 30.0
+150 150.0
+152 304.0
+153 153.0
+155 155.0
+156 156.0
+157 157.0
+158 158.0
+160 160.0
+162 162.0
+163 163.0
+164 328.0
+165 330.0
+166 166.0
+167 501.0
+168 168.0
+169 676.0
+17 17.0
+170 170.0
+172 344.0
+174 348.0
+175 350.0
+176 352.0
+177 177.0
+178 178.0
+179 358.0
+18 36.0
+180 180.0
+181 181.0
+183 183.0
+186 186.0
+187 561.0
+189 189.0
+19 19.0
+190 190.0
+191 382.0
+192 192.0
+193 579.0
+194 194.0
+195 390.0
+196 196.0
+197 394.0
+199 597.0
+2 2.0
+20 20.0
+200 400.0
+201 201.0
+202 202.0
+203 406.0
+205 410.0
+207 414.0
+208 624.0
+209 418.0
+213 426.0
+214 214.0
+216 432.0
+217 434.0
+218 218.0
+219 438.0
+221 442.0
+222 222.0
+223 446.0
+224 448.0
+226 226.0
+228 228.0
+229 458.0
+230 1150.0
+233 466.0
+235 235.0
+237 474.0
+238 476.0
+239 478.0
+24 48.0
+241 241.0
+242 484.0
+244 244.0
+247 247.0
+248 248.0
+249 249.0
+252 252.0
+255 510.0
+256 512.0
+257 257.0
+258 258.0
+26 52.0
+260 260.0
+262 262.0
+263 263.0
+265 530.0
+266 266.0
+27 27.0
+272 544.0
+273 819.0
+274 274.0
+275 275.0
+277 1108.0
+278 556.0
+28 28.0
+280 560.0
+281 562.0
+282 564.0
+283 283.0
+284 284.0
+285 285.0
+286 286.0
+287 287.0
+288 576.0
+289 289.0
+291 291.0
+292 292.0
+296 296.0
+298 894.0
+30 30.0
+302 302.0
+305 305.0
+306 306.0
+307 614.0
+308 308.0
+309 618.0
+310 310.0
+311 933.0
+315 315.0
+316 948.0
+317 634.0
+318 954.0
+321 642.0
+322 644.0
+323 323.0
+325 650.0
+327 981.0
+33 33.0
+331 662.0
+332 332.0
+333 666.0
+335 335.0
+336 336.0
+338 338.0
+339 339.0
+34 34.0
+341 341.0
+342 684.0
+344 688.0
+345 345.0
+348 1740.0
+35 105.0
+351 351.0
+353 706.0
+356 356.0
+360 360.0
+362 362.0
+364 364.0
+365 365.0
+366 366.0
+367 734.0
+368 368.0
+369 1107.0
+37 74.0
+373 373.0
+374 374.0
+375 375.0
+377 377.0
+378 378.0
+379 379.0
+382 764.0
+384 1152.0
+386 386.0
+389 389.0
+392 392.0
+393 393.0
+394 394.0
+395 790.0
+396 1188.0
+397 794.0
+399 798.0
+4 4.0
+400 400.0
+401 2005.0
+402 402.0
+403 1209.0
+404 808.0
+406 1624.0
+407 407.0
+409 1227.0
+41 41.0
+411 411.0
+413 826.0
+414 828.0
+417 1251.0
+418 418.0
+419 419.0
+42 84.0
+421 421.0
+424 848.0
+427 427.0
+429 858.0
+43 43.0
+430 1290.0
+431 1293.0
+432 432.0
+435 435.0
+436 436.0
+437 437.0
+438 1314.0
+439 878.0
+44 44.0
+443 443.0
+444 444.0
+446 446.0
+448 448.0
+449 449.0
+452 452.0
+453 453.0
+454 1362.0
+455 455.0
+457 457.0
+458 916.0
+459 918.0
+460 460.0
+462 924.0
+463 926.0
+466 1398.0
+467 467.0
+468 1872.0
+469 2345.0
+47 47.0
+470 470.0
+472 472.0
+475 475.0
+477 477.0
+478 956.0
+479 479.0
+480 1440.0
+481 481.0
+482 482.0
+483 483.0
+484 484.0
+485 485.0
+487 487.0
+489 1956.0
+490 490.0
+491 491.0
+492 984.0
+493 493.0
+494 494.0
+495 495.0
+496 496.0
+497 497.0
+498 1494.0
+5 15.0
+51 102.0
+53 53.0
+54 54.0
+57 57.0
+58 116.0
+64 64.0
+65 65.0
+66 66.0
+67 134.0
+69 69.0
+70 210.0
+72 144.0
+74 74.0
+76 152.0
+77 77.0
+78 78.0
+8 8.0
+80 80.0
+82 82.0
+83 166.0
+84 168.0
+85 85.0
+86 86.0
+87 87.0
+9 9.0
+90 270.0
+92 92.0
+95 190.0
+96 96.0
+97 194.0
+98 196.0
+PREHOOK: query: select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0 0.0 val_0
+10 10.0 val_10
+100 200.0 val_100
+103 206.0 val_103
+104 208.0 val_104
+105 105.0 val_105
+11 11.0 val_11
+111 111.0 val_111
+113 226.0 val_113
+114 114.0 val_114
+116 116.0 val_116
+118 236.0 val_118
+119 357.0 val_119
+12 24.0 val_12
+120 240.0 val_120
+125 250.0 val_125
+126 126.0 val_126
+128 384.0 val_128
+129 258.0 val_129
+131 131.0 val_131
+133 133.0 val_133
+134 268.0 val_134
+136 136.0 val_136
+137 274.0 val_137
+138 552.0 val_138
+143 143.0 val_143
+145 145.0 val_145
+146 292.0 val_146
+149 298.0 val_149
+15 30.0 val_15
+150 150.0 val_150
+152 304.0 val_152
+153 153.0 val_153
+155 155.0 val_155
+156 156.0 val_156
+157 157.0 val_157
+158 158.0 val_158
+160 160.0 val_160
+162 162.0 val_162
+163 163.0 val_163
+164 328.0 val_164
+165 330.0 val_165
+166 166.0 val_166
+167 501.0 val_167
+168 168.0 val_168
+169 676.0 val_169
+17 17.0 val_17
+170 170.0 val_170
+172 344.0 val_172
+174 348.0 val_174
+175 350.0 val_175
+176 352.0 val_176
+177 177.0 val_177
+178 178.0 val_178
+179 358.0 val_179
+18 36.0 val_18
+180 180.0 val_180
+181 181.0 val_181
+183 183.0 val_183
+186 186.0 val_186
+187 561.0 val_187
+189 189.0 val_189
+19 19.0 val_19
+190 190.0 val_190
+191 382.0 val_191
+192 192.0 val_192
+193 579.0 val_193
+194 194.0 val_194
+195 390.0 val_195
+196 196.0 val_196
+197 394.0 val_197
+199 597.0 val_199
+2 2.0 val_2
+20 20.0 val_20
+200 400.0 val_200
+201 201.0 val_201
+202 202.0 val_202
+203 406.0 val_203
+205 410.0 val_205
+207 414.0 val_207
+208 624.0 val_208
+209 418.0 val_209
+213 426.0 val_213
+214 214.0 val_214
+216 432.0 val_216
+217 434.0 val_217
+218 218.0 val_218
+219 438.0 val_219
+221 442.0 val_221
+222 222.0 val_222
+223 446.0 val_223
+224 448.0 val_224
+226 226.0 val_226
+228 228.0 val_228
+229 458.0 val_229
+230 1150.0 val_230
+233 466.0 val_233
+235 235.0 val_235
+237 474.0 val_237
+238 476.0 val_238
+239 478.0 val_239
+24 48.0 val_24
+241 241.0 val_241
+242 484.0 val_242
+244 244.0 val_244
+247 247.0 val_247
+248 248.0 val_248
+249 249.0 val_249
+252 252.0 val_252
+255 510.0 val_255
+256 512.0 val_256
+257 257.0 val_257
+258 258.0 val_258
+26 52.0 val_26
+260 260.0 val_260
+262 262.0 val_262
+263 263.0 val_263
+265 530.0 val_265
+266 266.0 val_266
+27 27.0 val_27
+272 544.0 val_272
+273 819.0 val_273
+274 274.0 val_274
+275 275.0 val_275
+277 1108.0 val_277
+278 556.0 val_278
+28 28.0 val_28
+280 560.0 val_280
+281 562.0 val_281
+282 564.0 val_282
+283 283.0 val_283
+284 284.0 val_284
+285 285.0 val_285
+286 286.0 val_286
+287 287.0 val_287
+288 576.0 val_288
+289 289.0 val_289
+291 291.0 val_291
+292 292.0 val_292
+296 296.0 val_296
+298 894.0 val_298
+30 30.0 val_30
+302 302.0 val_302
+305 305.0 val_305
+306 306.0 val_306
+307 614.0 val_307
+308 308.0 val_308
+309 618.0 val_309
+310 310.0 val_310
+311 933.0 val_311
+315 315.0 val_315
+316 948.0 val_316
+317 634.0 val_317
+318 954.0 val_318
+321 642.0 val_321
+322 644.0 val_322
+323 323.0 val_323
+325 650.0 val_325
+327 981.0 val_327
+33 33.0 val_33
+331 662.0 val_331
+332 332.0 val_332
+333 666.0 val_333
+335 335.0 val_335
+336 336.0 val_336
+338 338.0 val_338
+339 339.0 val_339
+34 34.0 val_34
+341 341.0 val_341
+342 684.0 val_342
+344 688.0 val_344
+345 345.0 val_345
+348 1740.0 val_348
+35 105.0 val_35
+351 351.0 val_351
+353 706.0 val_353
+356 356.0 val_356
+360 360.0 val_360
+362 362.0 val_362
+364 364.0 val_364
+365 365.0 val_365
+366 366.0 val_366
+367 734.0 val_367
+368 368.0 val_368
+369 1107.0 val_369
+37 74.0 val_37
+373 373.0 val_373
+374 374.0 val_374
+375 375.0 val_375
+377 377.0 val_377
+378 378.0 val_378
+379 379.0 val_379
+382 764.0 val_382
+384 1152.0 val_384
+386 386.0 val_386
+389 389.0 val_389
+392 392.0 val_392
+393 393.0 val_393
+394 394.0 val_394
+395 790.0 val_395
+396 1188.0 val_396
+397 794.0 val_397
+399 798.0 val_399
+4 4.0 val_4
+400 400.0 val_400
+401 2005.0 val_401
+402 402.0 val_402
+403 1209.0 val_403
+404 808.0 val_404
+406 1624.0 val_406
+407 407.0 val_407
+409 1227.0 val_409
+41 41.0 val_41
+411 411.0 val_411
+413 826.0 val_413
+414 828.0 val_414
+417 1251.0 val_417
+418 418.0 val_418
+419 419.0 val_419
+42 84.0 val_42
+421 421.0 val_421
+424 848.0 val_424
+427 427.0 val_427
+429 858.0 val_429
+43 43.0 val_43
+430 1290.0 val_430
+431 1293.0 val_431
+432 432.0 val_432
+435 435.0 val_435
+436 436.0 val_436
+437 437.0 val_437
+438 1314.0 val_438
+439 878.0 val_439
+44 44.0 val_44
+443 443.0 val_443
+444 444.0 val_444
+446 446.0 val_446
+448 448.0 val_448
+449 449.0 val_449
+452 452.0 val_452
+453 453.0 val_453
+454 1362.0 val_454
+455 455.0 val_455
+457 457.0 val_457
+458 916.0 val_458
+459 918.0 val_459
+460 460.0 val_460
+462 924.0 val_462
+463 926.0 val_463
+466 1398.0 val_466
+467 467.0 val_467
+468 1872.0 val_468
+469 2345.0 val_469
+47 47.0 val_47
+470 470.0 val_470
+472 472.0 val_472
+475 475.0 val_475
+477 477.0 val_477
+478 956.0 val_478
+479 479.0 val_479
+480 1440.0 val_480
+481 481.0 val_481
+482 482.0 val_482
+483 483.0 val_483
+484 484.0 val_484
+485 485.0 val_485
+487 487.0 val_487
+489 1956.0 val_489
+490 490.0 val_490
+491 491.0 val_491
+492 984.0 val_492
+493 493.0 val_493
+494 494.0 val_494
+495 495.0 val_495
+496 496.0 val_496
+497 497.0 val_497
+498 1494.0 val_498
+5 15.0 val_5
+51 102.0 val_51
+53 53.0 val_53
+54 54.0 val_54
+57 57.0 val_57
+58 116.0 val_58
+64 64.0 val_64
+65 65.0 val_65
+66 66.0 val_66
+67 134.0 val_67
+69 69.0 val_69
+70 210.0 val_70
+72 144.0 val_72
+74 74.0 val_74
+76 152.0 val_76
+77 77.0 val_77
+78 78.0 val_78
+8 8.0 val_8
+80 80.0 val_80
+82 82.0 val_82
+83 166.0 val_83
+84 168.0 val_84
+85 85.0 val_85
+86 86.0 val_86
+87 87.0 val_87
+9 9.0 val_9
+90 270.0 val_90
+92 92.0 val_92
+95 190.0 val_95
+96 96.0 val_96
+97 194.0 val_97
+98 196.0 val_98
+PREHOOK: query: select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0 0.0 NULL
+10 10.0 NULL
+100 200.0 NULL
+103 206.0 NULL
+104 208.0 NULL
+105 105.0 NULL
+11 11.0 NULL
+111 111.0 NULL
+113 226.0 NULL
+114 114.0 NULL
+116 116.0 NULL
+118 236.0 NULL
+119 357.0 NULL
+12 24.0 NULL
+120 240.0 NULL
+125 250.0 NULL
+126 126.0 NULL
+128 384.0 NULL
+129 258.0 NULL
+131 131.0 NULL
+133 133.0 NULL
+134 268.0 NULL
+136 136.0 NULL
+137 274.0 NULL
+138 552.0 NULL
+143 143.0 NULL
+145 145.0 NULL
+146 292.0 NULL
+149 298.0 NULL
+15 30.0 NULL
+150 150.0 NULL
+152 304.0 NULL
+153 153.0 NULL
+155 155.0 NULL
+156 156.0 NULL
+157 157.0 NULL
+158 158.0 NULL
+160 160.0 NULL
+162 162.0 NULL
+163 163.0 NULL
+164 328.0 NULL
+165 330.0 NULL
+166 166.0 NULL
+167 501.0 NULL
+168 168.0 NULL
+169 676.0 NULL
+17 17.0 NULL
+170 170.0 NULL
+172 344.0 NULL
+174 348.0 NULL
+175 350.0 NULL
+176 352.0 NULL
+177 177.0 NULL
+178 178.0 NULL
+179 358.0 NULL
+18 36.0 NULL
+180 180.0 NULL
+181 181.0 NULL
+183 183.0 NULL
+186 186.0 NULL
+187 561.0 NULL
+189 189.0 NULL
+19 19.0 NULL
+190 190.0 NULL
+191 382.0 NULL
+192 192.0 NULL
+193 579.0 NULL
+194 194.0 NULL
+195 390.0 NULL
+196 196.0 NULL
+197 394.0 NULL
+199 597.0 NULL
+2 2.0 NULL
+20 20.0 NULL
+200 400.0 NULL
+201 201.0 NULL
+202 202.0 NULL
+203 406.0 NULL
+205 410.0 NULL
+207 414.0 NULL
+208 624.0 NULL
+209 418.0 NULL
+213 426.0 NULL
+214 214.0 NULL
+216 432.0 NULL
+217 434.0 NULL
+218 218.0 NULL
+219 438.0 NULL
+221 442.0 NULL
+222 222.0 NULL
+223 446.0 NULL
+224 448.0 NULL
+226 226.0 NULL
+228 228.0 NULL
+229 458.0 NULL
+230 1150.0 NULL
+233 466.0 NULL
+235 235.0 NULL
+237 474.0 NULL
+238 476.0 NULL
+239 478.0 NULL
+24 48.0 NULL
+241 241.0 NULL
+242 484.0 NULL
+244 244.0 NULL
+247 247.0 NULL
+248 248.0 NULL
+249 249.0 NULL
+252 252.0 NULL
+255 510.0 NULL
+256 512.0 NULL
+257 257.0 NULL
+258 258.0 NULL
+26 52.0 NULL
+260 260.0 NULL
+262 262.0 NULL
+263 263.0 NULL
+265 530.0 NULL
+266 266.0 NULL
+27 27.0 NULL
+272 544.0 NULL
+273 819.0 NULL
+274 274.0 NULL
+275 275.0 NULL
+277 1108.0 NULL
+278 556.0 NULL
+28 28.0 NULL
+280 560.0 NULL
+281 562.0 NULL
+282 564.0 NULL
+283 283.0 NULL
+284 284.0 NULL
+285 285.0 NULL
+286 286.0 NULL
+287 287.0 NULL
+288 576.0 NULL
+289 289.0 NULL
+291 291.0 NULL
+292 292.0 NULL
+296 296.0 NULL
+298 894.0 NULL
+30 30.0 NULL
+302 302.0 NULL
+305 305.0 NULL
+306 306.0 NULL
+307 614.0 NULL
+308 308.0 NULL
+309 618.0 NULL
+310 310.0 NULL
+311 933.0 NULL
+315 315.0 NULL
+316 948.0 NULL
+317 634.0 NULL
+318 954.0 NULL
+321 642.0 NULL
+322 644.0 NULL
+323 323.0 NULL
+325 650.0 NULL
+327 981.0 NULL
+33 33.0 NULL
+331 662.0 NULL
+332 332.0 NULL
+333 666.0 NULL
+335 335.0 NULL
+336 336.0 NULL
+338 338.0 NULL
+339 339.0 NULL
+34 34.0 NULL
+341 341.0 NULL
+342 684.0 NULL
+344 688.0 NULL
+345 345.0 NULL
+348 1740.0 NULL
+35 105.0 NULL
+351 351.0 NULL
+353 706.0 NULL
+356 356.0 NULL
+360 360.0 NULL
+362 362.0 NULL
+364 364.0 NULL
+365 365.0 NULL
+366 366.0 NULL
+367 734.0 NULL
+368 368.0 NULL
+369 1107.0 NULL
+37 74.0 NULL
+373 373.0 NULL
+374 374.0 NULL
+375 375.0 NULL
+377 377.0 NULL
+378 378.0 NULL
+379 379.0 NULL
+382 764.0 NULL
+384 1152.0 NULL
+386 386.0 NULL
+389 389.0 NULL
+392 392.0 NULL
+393 393.0 NULL
+394 394.0 NULL
+395 790.0 NULL
+396 1188.0 NULL
+397 794.0 NULL
+399 798.0 NULL
+4 4.0 NULL
+400 400.0 NULL
+401 2005.0 NULL
+402 402.0 NULL
+403 1209.0 NULL
+404 808.0 NULL
+406 1624.0 NULL
+407 407.0 NULL
+409 1227.0 NULL
+41 41.0 NULL
+411 411.0 NULL
+413 826.0 NULL
+414 828.0 NULL
+417 1251.0 NULL
+418 418.0 NULL
+419 419.0 NULL
+42 84.0 NULL
+421 421.0 NULL
+424 848.0 NULL
+427 427.0 NULL
+429 858.0 NULL
+43 43.0 NULL
+430 1290.0 NULL
+431 1293.0 NULL
+432 432.0 NULL
+435 435.0 NULL
+436 436.0 NULL
+437 437.0 NULL
+438 1314.0 NULL
+439 878.0 NULL
+44 44.0 NULL
+443 443.0 NULL
+444 444.0 NULL
+446 446.0 NULL
+448 448.0 NULL
+449 449.0 NULL
+452 452.0 NULL
+453 453.0 NULL
+454 1362.0 NULL
+455 455.0 NULL
+457 457.0 NULL
+458 916.0 NULL
+459 918.0 NULL
+460 460.0 NULL
+462 924.0 NULL
+463 926.0 NULL
+466 1398.0 NULL
+467 467.0 NULL
+468 1872.0 NULL
+469 2345.0 NULL
+47 47.0 NULL
+470 470.0 NULL
+472 472.0 NULL
+475 475.0 NULL
+477 477.0 NULL
+478 956.0 NULL
+479 479.0 NULL
+480 1440.0 NULL
+481 481.0 NULL
+482 482.0 NULL
+483 483.0 NULL
+484 484.0 NULL
+485 485.0 NULL
+487 487.0 NULL
+489 1956.0 NULL
+490 490.0 NULL
+491 491.0 NULL
+492 984.0 NULL
+493 493.0 NULL
+494 494.0 NULL
+495 495.0 NULL
+496 496.0 NULL
+497 497.0 NULL
+498 1494.0 NULL
+5 15.0 NULL
+51 102.0 NULL
+53 53.0 NULL
+54 54.0 NULL
+57 57.0 NULL
+58 116.0 NULL
+64 64.0 NULL
+65 65.0 NULL
+66 66.0 NULL
+67 134.0 NULL
+69 69.0 NULL
+70 210.0 NULL
+72 144.0 NULL
+74 74.0 NULL
+76 152.0 NULL
+77 77.0 NULL
+78 78.0 NULL
+8 8.0 NULL
+80 80.0 NULL
+82 82.0 NULL
+83 166.0 NULL
+84 168.0 NULL
+85 85.0 NULL
+86 86.0 NULL
+87 87.0 NULL
+9 9.0 NULL
+90 270.0 NULL
+92 92.0 NULL
+95 190.0 NULL
+96 96.0 NULL
+97 194.0 NULL
+98 196.0 NULL
+PREHOOK: query: select key, sum(key) as value from src group by key order by key, value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, sum(key) as value from src group by key order by key, value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0 0.0
+10 10.0
+100 200.0
+103 206.0
+104 208.0
+105 105.0
+11 11.0
+111 111.0
+113 226.0
+114 114.0
+116 116.0
+118 236.0
+119 357.0
+12 24.0
+120 240.0
+125 250.0
+126 126.0
+128 384.0
+129 258.0
+131 131.0
+133 133.0
+134 268.0
+136 136.0
+137 274.0
+138 552.0
+143 143.0
+145 145.0
+146 292.0
+149 298.0
+15 30.0
+150 150.0
+152 304.0
+153 153.0
+155 155.0
+156 156.0
+157 157.0
+158 158.0
+160 160.0
+162 162.0
+163 163.0
+164 328.0
+165 330.0
+166 166.0
+167 501.0
+168 168.0
+169 676.0
+17 17.0
+170 170.0
+172 344.0
+174 348.0
+175 350.0
+176 352.0
+177 177.0
+178 178.0
+179 358.0
+18 36.0
+180 180.0
+181 181.0
+183 183.0
+186 186.0
+187 561.0
+189 189.0
+19 19.0
+190 190.0
+191 382.0
+192 192.0
+193 579.0
+194 194.0
+195 390.0
+196 196.0
+197 394.0
+199 597.0
+2 2.0
+20 20.0
+200 400.0
+201 201.0
+202 202.0
+203 406.0
+205 410.0
+207 414.0
+208 624.0
+209 418.0
+213 426.0
+214 214.0
+216 432.0
+217 434.0
+218 218.0
+219 438.0
+221 442.0
+222 222.0
+223 446.0
+224 448.0
+226 226.0
+228 228.0
+229 458.0
+230 1150.0
+233 466.0
+235 235.0
+237 474.0
+238 476.0
+239 478.0
+24 48.0
+241 241.0
+242 484.0
+244 244.0
+247 247.0
+248 248.0
+249 249.0
+252 252.0
+255 510.0
+256 512.0
+257 257.0
+258 258.0
+26 52.0
+260 260.0
+262 262.0
+263 263.0
+265 530.0
+266 266.0
+27 27.0
+272 544.0
+273 819.0
+274 274.0
+275 275.0
+277 1108.0
+278 556.0
+28 28.0
+280 560.0
+281 562.0
+282 564.0
+283 283.0
+284 284.0
+285 285.0
+286 286.0
+287 287.0
+288 576.0
+289 289.0
+291 291.0
+292 292.0
+296 296.0
+298 894.0
+30 30.0
+302 302.0
+305 305.0
+306 306.0
+307 614.0
+308 308.0
+309 618.0
+310 310.0
+311 933.0
+315 315.0
+316 948.0
+317 634.0
+318 954.0
+321 642.0
+322 644.0
+323 323.0
+325 650.0
+327 981.0
+33 33.0
+331 662.0
+332 332.0
+333 666.0
+335 335.0
+336 336.0
+338 338.0
+339 339.0
+34 34.0
+341 341.0
+342 684.0
+344 688.0
+345 345.0
+348 1740.0
+35 105.0
+351 351.0
+353 706.0
+356 356.0
+360 360.0
+362 362.0
+364 364.0
+365 365.0
+366 366.0
+367 734.0
+368 368.0
+369 1107.0
+37 74.0
+373 373.0
+374 374.0
+375 375.0
+377 377.0
+378 378.0
+379 379.0
+382 764.0
+384 1152.0
+386 386.0
+389 389.0
+392 392.0
+393 393.0
+394 394.0
+395 790.0
+396 1188.0
+397 794.0
+399 798.0
+4 4.0
+400 400.0
+401 2005.0
+402 402.0
+403 1209.0
+404 808.0
+406 1624.0
+407 407.0
+409 1227.0
+41 41.0
+411 411.0
+413 826.0
+414 828.0
+417 1251.0
+418 418.0
+419 419.0
+42 84.0
+421 421.0
+424 848.0
+427 427.0
+429 858.0
+43 43.0
+430 1290.0
+431 1293.0
+432 432.0
+435 435.0
+436 436.0
+437 437.0
+438 1314.0
+439 878.0
+44 44.0
+443 443.0
+444 444.0
+446 446.0
+448 448.0
+449 449.0
+452 452.0
+453 453.0
+454 1362.0
+455 455.0
+457 457.0
+458 916.0
+459 918.0
+460 460.0
+462 924.0
+463 926.0
+466 1398.0
+467 467.0
+468 1872.0
+469 2345.0
+47 47.0
+470 470.0
+472 472.0
+475 475.0
+477 477.0
+478 956.0
+479 479.0
+480 1440.0
+481 481.0
+482 482.0
+483 483.0
+484 484.0
+485 485.0
+487 487.0
+489 1956.0
+490 490.0
+491 491.0
+492 984.0
+493 493.0
+494 494.0
+495 495.0
+496 496.0
+497 497.0
+498 1494.0
+5 15.0
+51 102.0
+53 53.0
+54 54.0
+57 57.0
+58 116.0
+64 64.0
+65 65.0
+66 66.0
+67 134.0
+69 69.0
+70 210.0
+72 144.0
+74 74.0
+76 152.0
+77 77.0
+78 78.0
+8 8.0
+80 80.0
+82 82.0
+83 166.0
+84 168.0
+85 85.0
+86 86.0
+87 87.0
+9 9.0
+90 270.0
+92 92.0
+95 190.0
+96 96.0
+97 194.0
+98 196.0
+PREHOOK: query: select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+128 384.0
+146 292.0
+150 150.0
+213 426.0
+224 448.0
+238 476.0
+255 510.0
+273 819.0
+278 556.0
+311 933.0
+369 1107.0
+401 2005.0
+406 1624.0
+66 66.0
+98 196.0
+PREHOOK: query: select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+128 val_128
+128 val_128
+128 val_128
+146 val_146
+146 val_146
+150 val_150
+213 val_213
+213 val_213
+224 val_224
+224 val_224
+238 val_238
+238 val_238
+255 val_255
+255 val_255
+273 val_273
+273 val_273
+273 val_273
+278 val_278
+278 val_278
+311 val_311
+311 val_311
+311 val_311
+369 val_369
+369 val_369
+369 val_369
+401 val_401
+401 val_401
+401 val_401
+401 val_401
+401 val_401
+406 val_406
+406 val_406
+406 val_406
+406 val_406
+66 val_66
+98 val_98
+98 val_98
+PREHOOK: query: from (select key, value from src group by key, value) s select s.key group by s.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: from (select key, value from src group by key, value) s select s.key group by s.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0
+10
+100
+103
+104
+105
+11
+111
+113
+114
+116
+118
+119
+12
+120
+125
+126
+128
+129
+131
+133
+134
+136
+137
+138
+143
+145
+146
+149
+15
+150
+152
+153
+155
+156
+157
+158
+160
+162
+163
+164
+165
+166
+167
+168
+169
+17
+170
+172
+174
+175
+176
+177
+178
+179
+18
+180
+181
+183
+186
+187
+189
+19
+190
+191
+192
+193
+194
+195
+196
+197
+199
+2
+20
+200
+201
+202
+203
+205
+207
+208
+209
+213
+214
+216
+217
+218
+219
+221
+222
+223
+224
+226
+228
+229
+230
+233
+235
+237
+238
+239
+24
+241
+242
+244
+247
+248
+249
+252
+255
+256
+257
+258
+26
+260
+262
+263
+265
+266
+27
+272
+273
+274
+275
+277
+278
+28
+280
+281
+282
+283
+284
+285
+286
+287
+288
+289
+291
+292
+296
+298
+30
+302
+305
+306
+307
+308
+309
+310
+311
+315
+316
+317
+318
+321
+322
+323
+325
+327
+33
+331
+332
+333
+335
+336
+338
+339
+34
+341
+342
+344
+345
+348
+35
+351
+353
+356
+360
+362
+364
+365
+366
+367
+368
+369
+37
+373
+374
+375
+377
+378
+379
+382
+384
+386
+389
+392
+393
+394
+395
+396
+397
+399
+4
+400
+401
+402
+403
+404
+406
+407
+409
+41
+411
+413
+414
+417
+418
+419
+42
+421
+424
+427
+429
+43
+430
+431
+432
+435
+436
+437
+438
+439
+44
+443
+444
+446
+448
+449
+452
+453
+454
+455
+457
+458
+459
+460
+462
+463
+466
+467
+468
+469
+47
+470
+472
+475
+477
+478
+479
+480
+481
+482
+483
+484
+485
+487
+489
+490
+491
+492
+493
+494
+495
+496
+497
+498
+5
+51
+53
+54
+57
+58
+64
+65
+66
+67
+69
+70
+72
+74
+76
+77
+78
+8
+80
+82
+83
+84
+85
+86
+87
+9
+90
+92
+95
+96
+97
+98
+PREHOOK: query: explain select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_DISTRIBUTEBY (TOK_TABLE_OR_COL key)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL value))))) Q1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL key)))) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ q1:src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ Reduce Operator Tree:
+ Extract
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Group By Operator
+ aggregations:
+ expr: sum(_col0)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ mode: complete
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) Q1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL key))) (TOK_SELEXPR (TOK_FUNCTION lower (TOK_TABLE_OR_COL value)))) (TOK_GROUPBY (TOK_TABLE_OR_COL key) (TOK_FUNCTION lower (TOK_TABLE_OR_COL value)))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ q1:src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: lower(_col1)
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: lower(_col1)
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ Reduce Operator Tree:
+ Extract
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Group By Operator
+ aggregations:
+ expr: sum(_col0)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ expr: lower(_col1)
+ type: string
+ mode: complete
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col2
+ type: double
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1)
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (+ (TOK_TABLE_OR_COL value) 1) X)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key))))) Q1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL key))) (TOK_SELEXPR (+ (TOK_TABLE_OR_COL X) 1))) (TOK_GROUPBY (TOK_TABLE_OR_COL key) (+ (TOK_TABLE_OR_COL X) 1))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ q1:src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: (value + 1)
+ type: double
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: (_col1 + 1)
+ type: double
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: (_col1 + 1)
+ type: double
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ Reduce Operator Tree:
+ Extract
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ outputColumnNames: _col0, _col1
+ Group By Operator
+ aggregations:
+ expr: sum(_col0)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ expr: (_col1 + 1)
+ type: double
+ mode: complete
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col2
+ type: double
+ expr: _col1
+ type: double
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select key, sum(key) as value from src group by key order by key, value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select key, sum(key) as value from src group by key order by key, value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION sum (TOK_TABLE_OR_COL key)) value)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL key)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL value)))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: -1
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: sum(KEY._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: complete
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ sort order: ++
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: double
+ Reduce Operator Tree:
+ Extract
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src)) (TOK_TABREF (TOK_TABNAME src1)) (= (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL src1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key)) (TOK_SELEXPR (TOK_FUNCTION sum (. (TOK_TABLE_OR_COL src) key)))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL src) value))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ src
+ TableScan
+ alias: src
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ src1
+ TableScan
+ alias: src1
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ tag: -1
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: sum(KEY._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ expr: KEY._col1
+ type: string
+ mode: complete
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col2
+ type: double
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src)) (TOK_TABREF (TOK_TABNAME src1)) (= (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL src1) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value))) (TOK_ORDERBY (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src) key)) (TOK_TABSORTCOLNAMEASC (. (TOK_TABLE_OR_COL src) value)))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ src
+ TableScan
+ alias: src
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ src1
+ TableScan
+ alias: src1
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ tag: -1
+ value expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ Reduce Operator Tree:
+ Extract
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: explain from (select key, value from src group by key, value) s select s.key group by s.key
+PREHOOK: type: QUERY
+POSTHOOK: query: explain from (select key, value from src group by key, value) s select s.key group by s.key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_TABLE_OR_COL value))) (TOK_GROUPBY (TOK_TABLE_OR_COL key) (TOK_TABLE_OR_COL value)))) s)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL s) key))) (TOK_GROUPBY (. (TOK_TABLE_OR_COL s) key))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ s:src
+ TableScan
+ alias: src
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ outputColumnNames: key, value
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ expr: value
+ type: string
+ tag: -1
+ Reduce Operator Tree:
+ Group By Operator
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ expr: KEY._col1
+ type: string
+ mode: complete
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Group By Operator
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ mode: complete
+ outputColumnNames: _col0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, sum(key) from (select * from src distribute by key sort by key, value) Q1 group by key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0 0.0
+10 10.0
+100 200.0
+103 206.0
+104 208.0
+105 105.0
+11 11.0
+111 111.0
+113 226.0
+114 114.0
+116 116.0
+118 236.0
+119 357.0
+12 24.0
+120 240.0
+125 250.0
+126 126.0
+128 384.0
+129 258.0
+131 131.0
+133 133.0
+134 268.0
+136 136.0
+137 274.0
+138 552.0
+143 143.0
+145 145.0
+146 292.0
+149 298.0
+15 30.0
+150 150.0
+152 304.0
+153 153.0
+155 155.0
+156 156.0
+157 157.0
+158 158.0
+160 160.0
+162 162.0
+163 163.0
+164 328.0
+165 330.0
+166 166.0
+167 501.0
+168 168.0
+169 676.0
+17 17.0
+170 170.0
+172 344.0
+174 348.0
+175 350.0
+176 352.0
+177 177.0
+178 178.0
+179 358.0
+18 36.0
+180 180.0
+181 181.0
+183 183.0
+186 186.0
+187 561.0
+189 189.0
+19 19.0
+190 190.0
+191 382.0
+192 192.0
+193 579.0
+194 194.0
+195 390.0
+196 196.0
+197 394.0
+199 597.0
+2 2.0
+20 20.0
+200 400.0
+201 201.0
+202 202.0
+203 406.0
+205 410.0
+207 414.0
+208 624.0
+209 418.0
+213 426.0
+214 214.0
+216 432.0
+217 434.0
+218 218.0
+219 438.0
+221 442.0
+222 222.0
+223 446.0
+224 448.0
+226 226.0
+228 228.0
+229 458.0
+230 1150.0
+233 466.0
+235 235.0
+237 474.0
+238 476.0
+239 478.0
+24 48.0
+241 241.0
+242 484.0
+244 244.0
+247 247.0
+248 248.0
+249 249.0
+252 252.0
+255 510.0
+256 512.0
+257 257.0
+258 258.0
+26 52.0
+260 260.0
+262 262.0
+263 263.0
+265 530.0
+266 266.0
+27 27.0
+272 544.0
+273 819.0
+274 274.0
+275 275.0
+277 1108.0
+278 556.0
+28 28.0
+280 560.0
+281 562.0
+282 564.0
+283 283.0
+284 284.0
+285 285.0
+286 286.0
+287 287.0
+288 576.0
+289 289.0
+291 291.0
+292 292.0
+296 296.0
+298 894.0
+30 30.0
+302 302.0
+305 305.0
+306 306.0
+307 614.0
+308 308.0
+309 618.0
+310 310.0
+311 933.0
+315 315.0
+316 948.0
+317 634.0
+318 954.0
+321 642.0
+322 644.0
+323 323.0
+325 650.0
+327 981.0
+33 33.0
+331 662.0
+332 332.0
+333 666.0
+335 335.0
+336 336.0
+338 338.0
+339 339.0
+34 34.0
+341 341.0
+342 684.0
+344 688.0
+345 345.0
+348 1740.0
+35 105.0
+351 351.0
+353 706.0
+356 356.0
+360 360.0
+362 362.0
+364 364.0
+365 365.0
+366 366.0
+367 734.0
+368 368.0
+369 1107.0
+37 74.0
+373 373.0
+374 374.0
+375 375.0
+377 377.0
+378 378.0
+379 379.0
+382 764.0
+384 1152.0
+386 386.0
+389 389.0
+392 392.0
+393 393.0
+394 394.0
+395 790.0
+396 1188.0
+397 794.0
+399 798.0
+4 4.0
+400 400.0
+401 2005.0
+402 402.0
+403 1209.0
+404 808.0
+406 1624.0
+407 407.0
+409 1227.0
+41 41.0
+411 411.0
+413 826.0
+414 828.0
+417 1251.0
+418 418.0
+419 419.0
+42 84.0
+421 421.0
+424 848.0
+427 427.0
+429 858.0
+43 43.0
+430 1290.0
+431 1293.0
+432 432.0
+435 435.0
+436 436.0
+437 437.0
+438 1314.0
+439 878.0
+44 44.0
+443 443.0
+444 444.0
+446 446.0
+448 448.0
+449 449.0
+452 452.0
+453 453.0
+454 1362.0
+455 455.0
+457 457.0
+458 916.0
+459 918.0
+460 460.0
+462 924.0
+463 926.0
+466 1398.0
+467 467.0
+468 1872.0
+469 2345.0
+47 47.0
+470 470.0
+472 472.0
+475 475.0
+477 477.0
+478 956.0
+479 479.0
+480 1440.0
+481 481.0
+482 482.0
+483 483.0
+484 484.0
+485 485.0
+487 487.0
+489 1956.0
+490 490.0
+491 491.0
+492 984.0
+493 493.0
+494 494.0
+495 495.0
+496 496.0
+497 497.0
+498 1494.0
+5 15.0
+51 102.0
+53 53.0
+54 54.0
+57 57.0
+58 116.0
+64 64.0
+65 65.0
+66 66.0
+67 134.0
+69 69.0
+70 210.0
+72 144.0
+74 74.0
+76 152.0
+77 77.0
+78 78.0
+8 8.0
+80 80.0
+82 82.0
+83 166.0
+84 168.0
+85 85.0
+86 86.0
+87 87.0
+9 9.0
+90 270.0
+92 92.0
+95 190.0
+96 96.0
+97 194.0
+98 196.0
+PREHOOK: query: select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, sum(key), lower(value) from (select * from src order by key) Q1 group by key, lower(value)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0 0.0 val_0
+10 10.0 val_10
+100 200.0 val_100
+103 206.0 val_103
+104 208.0 val_104
+105 105.0 val_105
+11 11.0 val_11
+111 111.0 val_111
+113 226.0 val_113
+114 114.0 val_114
+116 116.0 val_116
+118 236.0 val_118
+119 357.0 val_119
+12 24.0 val_12
+120 240.0 val_120
+125 250.0 val_125
+126 126.0 val_126
+128 384.0 val_128
+129 258.0 val_129
+131 131.0 val_131
+133 133.0 val_133
+134 268.0 val_134
+136 136.0 val_136
+137 274.0 val_137
+138 552.0 val_138
+143 143.0 val_143
+145 145.0 val_145
+146 292.0 val_146
+149 298.0 val_149
+15 30.0 val_15
+150 150.0 val_150
+152 304.0 val_152
+153 153.0 val_153
+155 155.0 val_155
+156 156.0 val_156
+157 157.0 val_157
+158 158.0 val_158
+160 160.0 val_160
+162 162.0 val_162
+163 163.0 val_163
+164 328.0 val_164
+165 330.0 val_165
+166 166.0 val_166
+167 501.0 val_167
+168 168.0 val_168
+169 676.0 val_169
+17 17.0 val_17
+170 170.0 val_170
+172 344.0 val_172
+174 348.0 val_174
+175 350.0 val_175
+176 352.0 val_176
+177 177.0 val_177
+178 178.0 val_178
+179 358.0 val_179
+18 36.0 val_18
+180 180.0 val_180
+181 181.0 val_181
+183 183.0 val_183
+186 186.0 val_186
+187 561.0 val_187
+189 189.0 val_189
+19 19.0 val_19
+190 190.0 val_190
+191 382.0 val_191
+192 192.0 val_192
+193 579.0 val_193
+194 194.0 val_194
+195 390.0 val_195
+196 196.0 val_196
+197 394.0 val_197
+199 597.0 val_199
+2 2.0 val_2
+20 20.0 val_20
+200 400.0 val_200
+201 201.0 val_201
+202 202.0 val_202
+203 406.0 val_203
+205 410.0 val_205
+207 414.0 val_207
+208 624.0 val_208
+209 418.0 val_209
+213 426.0 val_213
+214 214.0 val_214
+216 432.0 val_216
+217 434.0 val_217
+218 218.0 val_218
+219 438.0 val_219
+221 442.0 val_221
+222 222.0 val_222
+223 446.0 val_223
+224 448.0 val_224
+226 226.0 val_226
+228 228.0 val_228
+229 458.0 val_229
+230 1150.0 val_230
+233 466.0 val_233
+235 235.0 val_235
+237 474.0 val_237
+238 476.0 val_238
+239 478.0 val_239
+24 48.0 val_24
+241 241.0 val_241
+242 484.0 val_242
+244 244.0 val_244
+247 247.0 val_247
+248 248.0 val_248
+249 249.0 val_249
+252 252.0 val_252
+255 510.0 val_255
+256 512.0 val_256
+257 257.0 val_257
+258 258.0 val_258
+26 52.0 val_26
+260 260.0 val_260
+262 262.0 val_262
+263 263.0 val_263
+265 530.0 val_265
+266 266.0 val_266
+27 27.0 val_27
+272 544.0 val_272
+273 819.0 val_273
+274 274.0 val_274
+275 275.0 val_275
+277 1108.0 val_277
+278 556.0 val_278
+28 28.0 val_28
+280 560.0 val_280
+281 562.0 val_281
+282 564.0 val_282
+283 283.0 val_283
+284 284.0 val_284
+285 285.0 val_285
+286 286.0 val_286
+287 287.0 val_287
+288 576.0 val_288
+289 289.0 val_289
+291 291.0 val_291
+292 292.0 val_292
+296 296.0 val_296
+298 894.0 val_298
+30 30.0 val_30
+302 302.0 val_302
+305 305.0 val_305
+306 306.0 val_306
+307 614.0 val_307
+308 308.0 val_308
+309 618.0 val_309
+310 310.0 val_310
+311 933.0 val_311
+315 315.0 val_315
+316 948.0 val_316
+317 634.0 val_317
+318 954.0 val_318
+321 642.0 val_321
+322 644.0 val_322
+323 323.0 val_323
+325 650.0 val_325
+327 981.0 val_327
+33 33.0 val_33
+331 662.0 val_331
+332 332.0 val_332
+333 666.0 val_333
+335 335.0 val_335
+336 336.0 val_336
+338 338.0 val_338
+339 339.0 val_339
+34 34.0 val_34
+341 341.0 val_341
+342 684.0 val_342
+344 688.0 val_344
+345 345.0 val_345
+348 1740.0 val_348
+35 105.0 val_35
+351 351.0 val_351
+353 706.0 val_353
+356 356.0 val_356
+360 360.0 val_360
+362 362.0 val_362
+364 364.0 val_364
+365 365.0 val_365
+366 366.0 val_366
+367 734.0 val_367
+368 368.0 val_368
+369 1107.0 val_369
+37 74.0 val_37
+373 373.0 val_373
+374 374.0 val_374
+375 375.0 val_375
+377 377.0 val_377
+378 378.0 val_378
+379 379.0 val_379
+382 764.0 val_382
+384 1152.0 val_384
+386 386.0 val_386
+389 389.0 val_389
+392 392.0 val_392
+393 393.0 val_393
+394 394.0 val_394
+395 790.0 val_395
+396 1188.0 val_396
+397 794.0 val_397
+399 798.0 val_399
+4 4.0 val_4
+400 400.0 val_400
+401 2005.0 val_401
+402 402.0 val_402
+403 1209.0 val_403
+404 808.0 val_404
+406 1624.0 val_406
+407 407.0 val_407
+409 1227.0 val_409
+41 41.0 val_41
+411 411.0 val_411
+413 826.0 val_413
+414 828.0 val_414
+417 1251.0 val_417
+418 418.0 val_418
+419 419.0 val_419
+42 84.0 val_42
+421 421.0 val_421
+424 848.0 val_424
+427 427.0 val_427
+429 858.0 val_429
+43 43.0 val_43
+430 1290.0 val_430
+431 1293.0 val_431
+432 432.0 val_432
+435 435.0 val_435
+436 436.0 val_436
+437 437.0 val_437
+438 1314.0 val_438
+439 878.0 val_439
+44 44.0 val_44
+443 443.0 val_443
+444 444.0 val_444
+446 446.0 val_446
+448 448.0 val_448
+449 449.0 val_449
+452 452.0 val_452
+453 453.0 val_453
+454 1362.0 val_454
+455 455.0 val_455
+457 457.0 val_457
+458 916.0 val_458
+459 918.0 val_459
+460 460.0 val_460
+462 924.0 val_462
+463 926.0 val_463
+466 1398.0 val_466
+467 467.0 val_467
+468 1872.0 val_468
+469 2345.0 val_469
+47 47.0 val_47
+470 470.0 val_470
+472 472.0 val_472
+475 475.0 val_475
+477 477.0 val_477
+478 956.0 val_478
+479 479.0 val_479
+480 1440.0 val_480
+481 481.0 val_481
+482 482.0 val_482
+483 483.0 val_483
+484 484.0 val_484
+485 485.0 val_485
+487 487.0 val_487
+489 1956.0 val_489
+490 490.0 val_490
+491 491.0 val_491
+492 984.0 val_492
+493 493.0 val_493
+494 494.0 val_494
+495 495.0 val_495
+496 496.0 val_496
+497 497.0 val_497
+498 1494.0 val_498
+5 15.0 val_5
+51 102.0 val_51
+53 53.0 val_53
+54 54.0 val_54
+57 57.0 val_57
+58 116.0 val_58
+64 64.0 val_64
+65 65.0 val_65
+66 66.0 val_66
+67 134.0 val_67
+69 69.0 val_69
+70 210.0 val_70
+72 144.0 val_72
+74 74.0 val_74
+76 152.0 val_76
+77 77.0 val_77
+78 78.0 val_78
+8 8.0 val_8
+80 80.0 val_80
+82 82.0 val_82
+83 166.0 val_83
+84 168.0 val_84
+85 85.0 val_85
+86 86.0 val_86
+87 87.0 val_87
+9 9.0 val_9
+90 270.0 val_90
+92 92.0 val_92
+95 190.0 val_95
+96 96.0 val_96
+97 194.0 val_97
+98 196.0 val_98
+PREHOOK: query: select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, sum(key), (X + 1) from (select key, (value + 1) as X from src order by key) Q1 group by key, (X + 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0 0.0 NULL
+10 10.0 NULL
+100 200.0 NULL
+103 206.0 NULL
+104 208.0 NULL
+105 105.0 NULL
+11 11.0 NULL
+111 111.0 NULL
+113 226.0 NULL
+114 114.0 NULL
+116 116.0 NULL
+118 236.0 NULL
+119 357.0 NULL
+12 24.0 NULL
+120 240.0 NULL
+125 250.0 NULL
+126 126.0 NULL
+128 384.0 NULL
+129 258.0 NULL
+131 131.0 NULL
+133 133.0 NULL
+134 268.0 NULL
+136 136.0 NULL
+137 274.0 NULL
+138 552.0 NULL
+143 143.0 NULL
+145 145.0 NULL
+146 292.0 NULL
+149 298.0 NULL
+15 30.0 NULL
+150 150.0 NULL
+152 304.0 NULL
+153 153.0 NULL
+155 155.0 NULL
+156 156.0 NULL
+157 157.0 NULL
+158 158.0 NULL
+160 160.0 NULL
+162 162.0 NULL
+163 163.0 NULL
+164 328.0 NULL
+165 330.0 NULL
+166 166.0 NULL
+167 501.0 NULL
+168 168.0 NULL
+169 676.0 NULL
+17 17.0 NULL
+170 170.0 NULL
+172 344.0 NULL
+174 348.0 NULL
+175 350.0 NULL
+176 352.0 NULL
+177 177.0 NULL
+178 178.0 NULL
+179 358.0 NULL
+18 36.0 NULL
+180 180.0 NULL
+181 181.0 NULL
+183 183.0 NULL
+186 186.0 NULL
+187 561.0 NULL
+189 189.0 NULL
+19 19.0 NULL
+190 190.0 NULL
+191 382.0 NULL
+192 192.0 NULL
+193 579.0 NULL
+194 194.0 NULL
+195 390.0 NULL
+196 196.0 NULL
+197 394.0 NULL
+199 597.0 NULL
+2 2.0 NULL
+20 20.0 NULL
+200 400.0 NULL
+201 201.0 NULL
+202 202.0 NULL
+203 406.0 NULL
+205 410.0 NULL
+207 414.0 NULL
+208 624.0 NULL
+209 418.0 NULL
+213 426.0 NULL
+214 214.0 NULL
+216 432.0 NULL
+217 434.0 NULL
+218 218.0 NULL
+219 438.0 NULL
+221 442.0 NULL
+222 222.0 NULL
+223 446.0 NULL
+224 448.0 NULL
+226 226.0 NULL
+228 228.0 NULL
+229 458.0 NULL
+230 1150.0 NULL
+233 466.0 NULL
+235 235.0 NULL
+237 474.0 NULL
+238 476.0 NULL
+239 478.0 NULL
+24 48.0 NULL
+241 241.0 NULL
+242 484.0 NULL
+244 244.0 NULL
+247 247.0 NULL
+248 248.0 NULL
+249 249.0 NULL
+252 252.0 NULL
+255 510.0 NULL
+256 512.0 NULL
+257 257.0 NULL
+258 258.0 NULL
+26 52.0 NULL
+260 260.0 NULL
+262 262.0 NULL
+263 263.0 NULL
+265 530.0 NULL
+266 266.0 NULL
+27 27.0 NULL
+272 544.0 NULL
+273 819.0 NULL
+274 274.0 NULL
+275 275.0 NULL
+277 1108.0 NULL
+278 556.0 NULL
+28 28.0 NULL
+280 560.0 NULL
+281 562.0 NULL
+282 564.0 NULL
+283 283.0 NULL
+284 284.0 NULL
+285 285.0 NULL
+286 286.0 NULL
+287 287.0 NULL
+288 576.0 NULL
+289 289.0 NULL
+291 291.0 NULL
+292 292.0 NULL
+296 296.0 NULL
+298 894.0 NULL
+30 30.0 NULL
+302 302.0 NULL
+305 305.0 NULL
+306 306.0 NULL
+307 614.0 NULL
+308 308.0 NULL
+309 618.0 NULL
+310 310.0 NULL
+311 933.0 NULL
+315 315.0 NULL
+316 948.0 NULL
+317 634.0 NULL
+318 954.0 NULL
+321 642.0 NULL
+322 644.0 NULL
+323 323.0 NULL
+325 650.0 NULL
+327 981.0 NULL
+33 33.0 NULL
+331 662.0 NULL
+332 332.0 NULL
+333 666.0 NULL
+335 335.0 NULL
+336 336.0 NULL
+338 338.0 NULL
+339 339.0 NULL
+34 34.0 NULL
+341 341.0 NULL
+342 684.0 NULL
+344 688.0 NULL
+345 345.0 NULL
+348 1740.0 NULL
+35 105.0 NULL
+351 351.0 NULL
+353 706.0 NULL
+356 356.0 NULL
+360 360.0 NULL
+362 362.0 NULL
+364 364.0 NULL
+365 365.0 NULL
+366 366.0 NULL
+367 734.0 NULL
+368 368.0 NULL
+369 1107.0 NULL
+37 74.0 NULL
+373 373.0 NULL
+374 374.0 NULL
+375 375.0 NULL
+377 377.0 NULL
+378 378.0 NULL
+379 379.0 NULL
+382 764.0 NULL
+384 1152.0 NULL
+386 386.0 NULL
+389 389.0 NULL
+392 392.0 NULL
+393 393.0 NULL
+394 394.0 NULL
+395 790.0 NULL
+396 1188.0 NULL
+397 794.0 NULL
+399 798.0 NULL
+4 4.0 NULL
+400 400.0 NULL
+401 2005.0 NULL
+402 402.0 NULL
+403 1209.0 NULL
+404 808.0 NULL
+406 1624.0 NULL
+407 407.0 NULL
+409 1227.0 NULL
+41 41.0 NULL
+411 411.0 NULL
+413 826.0 NULL
+414 828.0 NULL
+417 1251.0 NULL
+418 418.0 NULL
+419 419.0 NULL
+42 84.0 NULL
+421 421.0 NULL
+424 848.0 NULL
+427 427.0 NULL
+429 858.0 NULL
+43 43.0 NULL
+430 1290.0 NULL
+431 1293.0 NULL
+432 432.0 NULL
+435 435.0 NULL
+436 436.0 NULL
+437 437.0 NULL
+438 1314.0 NULL
+439 878.0 NULL
+44 44.0 NULL
+443 443.0 NULL
+444 444.0 NULL
+446 446.0 NULL
+448 448.0 NULL
+449 449.0 NULL
+452 452.0 NULL
+453 453.0 NULL
+454 1362.0 NULL
+455 455.0 NULL
+457 457.0 NULL
+458 916.0 NULL
+459 918.0 NULL
+460 460.0 NULL
+462 924.0 NULL
+463 926.0 NULL
+466 1398.0 NULL
+467 467.0 NULL
+468 1872.0 NULL
+469 2345.0 NULL
+47 47.0 NULL
+470 470.0 NULL
+472 472.0 NULL
+475 475.0 NULL
+477 477.0 NULL
+478 956.0 NULL
+479 479.0 NULL
+480 1440.0 NULL
+481 481.0 NULL
+482 482.0 NULL
+483 483.0 NULL
+484 484.0 NULL
+485 485.0 NULL
+487 487.0 NULL
+489 1956.0 NULL
+490 490.0 NULL
+491 491.0 NULL
+492 984.0 NULL
+493 493.0 NULL
+494 494.0 NULL
+495 495.0 NULL
+496 496.0 NULL
+497 497.0 NULL
+498 1494.0 NULL
+5 15.0 NULL
+51 102.0 NULL
+53 53.0 NULL
+54 54.0 NULL
+57 57.0 NULL
+58 116.0 NULL
+64 64.0 NULL
+65 65.0 NULL
+66 66.0 NULL
+67 134.0 NULL
+69 69.0 NULL
+70 210.0 NULL
+72 144.0 NULL
+74 74.0 NULL
+76 152.0 NULL
+77 77.0 NULL
+78 78.0 NULL
+8 8.0 NULL
+80 80.0 NULL
+82 82.0 NULL
+83 166.0 NULL
+84 168.0 NULL
+85 85.0 NULL
+86 86.0 NULL
+87 87.0 NULL
+9 9.0 NULL
+90 270.0 NULL
+92 92.0 NULL
+95 190.0 NULL
+96 96.0 NULL
+97 194.0 NULL
+98 196.0 NULL
+PREHOOK: query: select key, sum(key) as value from src group by key order by key, value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, sum(key) as value from src group by key order by key, value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0 0.0
+10 10.0
+100 200.0
+103 206.0
+104 208.0
+105 105.0
+11 11.0
+111 111.0
+113 226.0
+114 114.0
+116 116.0
+118 236.0
+119 357.0
+12 24.0
+120 240.0
+125 250.0
+126 126.0
+128 384.0
+129 258.0
+131 131.0
+133 133.0
+134 268.0
+136 136.0
+137 274.0
+138 552.0
+143 143.0
+145 145.0
+146 292.0
+149 298.0
+15 30.0
+150 150.0
+152 304.0
+153 153.0
+155 155.0
+156 156.0
+157 157.0
+158 158.0
+160 160.0
+162 162.0
+163 163.0
+164 328.0
+165 330.0
+166 166.0
+167 501.0
+168 168.0
+169 676.0
+17 17.0
+170 170.0
+172 344.0
+174 348.0
+175 350.0
+176 352.0
+177 177.0
+178 178.0
+179 358.0
+18 36.0
+180 180.0
+181 181.0
+183 183.0
+186 186.0
+187 561.0
+189 189.0
+19 19.0
+190 190.0
+191 382.0
+192 192.0
+193 579.0
+194 194.0
+195 390.0
+196 196.0
+197 394.0
+199 597.0
+2 2.0
+20 20.0
+200 400.0
+201 201.0
+202 202.0
+203 406.0
+205 410.0
+207 414.0
+208 624.0
+209 418.0
+213 426.0
+214 214.0
+216 432.0
+217 434.0
+218 218.0
+219 438.0
+221 442.0
+222 222.0
+223 446.0
+224 448.0
+226 226.0
+228 228.0
+229 458.0
+230 1150.0
+233 466.0
+235 235.0
+237 474.0
+238 476.0
+239 478.0
+24 48.0
+241 241.0
+242 484.0
+244 244.0
+247 247.0
+248 248.0
+249 249.0
+252 252.0
+255 510.0
+256 512.0
+257 257.0
+258 258.0
+26 52.0
+260 260.0
+262 262.0
+263 263.0
+265 530.0
+266 266.0
+27 27.0
+272 544.0
+273 819.0
+274 274.0
+275 275.0
+277 1108.0
+278 556.0
+28 28.0
+280 560.0
+281 562.0
+282 564.0
+283 283.0
+284 284.0
+285 285.0
+286 286.0
+287 287.0
+288 576.0
+289 289.0
+291 291.0
+292 292.0
+296 296.0
+298 894.0
+30 30.0
+302 302.0
+305 305.0
+306 306.0
+307 614.0
+308 308.0
+309 618.0
+310 310.0
+311 933.0
+315 315.0
+316 948.0
+317 634.0
+318 954.0
+321 642.0
+322 644.0
+323 323.0
+325 650.0
+327 981.0
+33 33.0
+331 662.0
+332 332.0
+333 666.0
+335 335.0
+336 336.0
+338 338.0
+339 339.0
+34 34.0
+341 341.0
+342 684.0
+344 688.0
+345 345.0
+348 1740.0
+35 105.0
+351 351.0
+353 706.0
+356 356.0
+360 360.0
+362 362.0
+364 364.0
+365 365.0
+366 366.0
+367 734.0
+368 368.0
+369 1107.0
+37 74.0
+373 373.0
+374 374.0
+375 375.0
+377 377.0
+378 378.0
+379 379.0
+382 764.0
+384 1152.0
+386 386.0
+389 389.0
+392 392.0
+393 393.0
+394 394.0
+395 790.0
+396 1188.0
+397 794.0
+399 798.0
+4 4.0
+400 400.0
+401 2005.0
+402 402.0
+403 1209.0
+404 808.0
+406 1624.0
+407 407.0
+409 1227.0
+41 41.0
+411 411.0
+413 826.0
+414 828.0
+417 1251.0
+418 418.0
+419 419.0
+42 84.0
+421 421.0
+424 848.0
+427 427.0
+429 858.0
+43 43.0
+430 1290.0
+431 1293.0
+432 432.0
+435 435.0
+436 436.0
+437 437.0
+438 1314.0
+439 878.0
+44 44.0
+443 443.0
+444 444.0
+446 446.0
+448 448.0
+449 449.0
+452 452.0
+453 453.0
+454 1362.0
+455 455.0
+457 457.0
+458 916.0
+459 918.0
+460 460.0
+462 924.0
+463 926.0
+466 1398.0
+467 467.0
+468 1872.0
+469 2345.0
+47 47.0
+470 470.0
+472 472.0
+475 475.0
+477 477.0
+478 956.0
+479 479.0
+480 1440.0
+481 481.0
+482 482.0
+483 483.0
+484 484.0
+485 485.0
+487 487.0
+489 1956.0
+490 490.0
+491 491.0
+492 984.0
+493 493.0
+494 494.0
+495 495.0
+496 496.0
+497 497.0
+498 1494.0
+5 15.0
+51 102.0
+53 53.0
+54 54.0
+57 57.0
+58 116.0
+64 64.0
+65 65.0
+66 66.0
+67 134.0
+69 69.0
+70 210.0
+72 144.0
+74 74.0
+76 152.0
+77 77.0
+78 78.0
+8 8.0
+80 80.0
+82 82.0
+83 166.0
+84 168.0
+85 85.0
+86 86.0
+87 87.0
+9 9.0
+90 270.0
+92 92.0
+95 190.0
+96 96.0
+97 194.0
+98 196.0
+PREHOOK: query: select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: select src.key, sum(src.key) FROM src JOIN src1 ON src.key = src1.key group by src.key, src.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+128 384.0
+146 292.0
+150 150.0
+213 426.0
+224 448.0
+238 476.0
+255 510.0
+273 819.0
+278 556.0
+311 933.0
+369 1107.0
+401 2005.0
+406 1624.0
+66 66.0
+98 196.0
+PREHOOK: query: select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+POSTHOOK: query: select src.key, src.value FROM src JOIN src1 ON src.key = src1.key order by src.key, src.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Input: default@src1
+#### A masked pattern was here ####
+128 val_128
+128 val_128
+128 val_128
+146 val_146
+146 val_146
+150 val_150
+213 val_213
+213 val_213
+224 val_224
+224 val_224
+238 val_238
+238 val_238
+255 val_255
+255 val_255
+273 val_273
+273 val_273
+273 val_273
+278 val_278
+278 val_278
+311 val_311
+311 val_311
+311 val_311
+369 val_369
+369 val_369
+369 val_369
+401 val_401
+401 val_401
+401 val_401
+401 val_401
+401 val_401
+406 val_406
+406 val_406
+406 val_406
+406 val_406
+66 val_66
+98 val_98
+98 val_98
+PREHOOK: query: from (select key, value from src group by key, value) s select s.key group by s.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: from (select key, value from src group by key, value) s select s.key group by s.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0
+10
+100
+103
+104
+105
+11
+111
+113
+114
+116
+118
+119
+12
+120
+125
+126
+128
+129
+131
+133
+134
+136
+137
+138
+143
+145
+146
+149
+15
+150
+152
+153
+155
+156
+157
+158
+160
+162
+163
+164
+165
+166
+167
+168
+169
+17
+170
+172
+174
+175
+176
+177
+178
+179
+18
+180
+181
+183
+186
+187
+189
+19
+190
+191
+192
+193
+194
+195
+196
+197
+199
+2
+20
+200
+201
+202
+203
+205
+207
+208
+209
+213
+214
+216
+217
+218
+219
+221
+222
+223
+224
+226
+228
+229
+230
+233
+235
+237
+238
+239
+24
+241
+242
+244
+247
+248
+249
+252
+255
+256
+257
+258
+26
+260
+262
+263
+265
+266
+27
+272
+273
+274
+275
+277
+278
+28
+280
+281
+282
+283
+284
+285
+286
+287
+288
+289
+291
+292
+296
+298
+30
+302
+305
+306
+307
+308
+309
+310
+311
+315
+316
+317
+318
+321
+322
+323
+325
+327
+33
+331
+332
+333
+335
+336
+338
+339
+34
+341
+342
+344
+345
+348
+35
+351
+353
+356
+360
+362
+364
+365
+366
+367
+368
+369
+37
+373
+374
+375
+377
+378
+379
+382
+384
+386
+389
+392
+393
+394
+395
+396
+397
+399
+4
+400
+401
+402
+403
+404
+406
+407
+409
+41
+411
+413
+414
+417
+418
+419
+42
+421
+424
+427
+429
+43
+430
+431
+432
+435
+436
+437
+438
+439
+44
+443
+444
+446
+448
+449
+452
+453
+454
+455
+457
+458
+459
+460
+462
+463
+466
+467
+468
+469
+47
+470
+472
+475
+477
+478
+479
+480
+481
+482
+483
+484
+485
+487
+489
+490
+491
+492
+493
+494
+495
+496
+497
+498
+5
+51
+53
+54
+57
+58
+64
+65
+66
+67
+69
+70
+72
+74
+76
+77
+78
+8
+80
+82
+83
+84
+85
+86
+87
+9
+90
+92
+95
+96
+97
+98
diff --git ql/src/test/results/clientpositive/semijoin.q.out ql/src/test/results/clientpositive/semijoin.q.out
index e60513c..1671625 100644
--- ql/src/test/results/clientpositive/semijoin.q.out
+++ ql/src/test/results/clientpositive/semijoin.q.out
@@ -1554,7 +1554,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -1630,38 +1629,19 @@ STAGE PLANS:
expr: _col1
type: string
outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: int
- expr: _col1
- type: string
- sort order: ++
- tag: -1
- value expressions:
+ Select Operator
+ expressions:
expr: _col0
type: int
expr: _col1
type: string
- Reduce Operator Tree:
- Extract
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
@@ -1859,7 +1839,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -1933,34 +1912,17 @@ STAGE PLANS:
expr: _col0
type: int
outputColumnNames: _col0
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: int
- sort order: +
- tag: -1
- value expressions:
+ Select Operator
+ expressions:
expr: _col0
type: int
- Reduce Operator Tree:
- Extract
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
@@ -2330,7 +2292,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
Stage-0 is a root stage
STAGE PLANS:
@@ -2404,34 +2365,17 @@ STAGE PLANS:
expr: _col0
type: int
outputColumnNames: _col0
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-
- Stage: Stage-2
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: int
- sort order: +
- tag: -1
- value expressions:
+ Select Operator
+ expressions:
expr: _col0
type: int
- Reduce Operator Tree:
- Extract
- File Output Operator
- compressed: false
- GlobalTableId: 0
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Stage: Stage-0
Fetch Operator
diff --git ql/src/test/results/clientpositive/union24.q.out ql/src/test/results/clientpositive/union24.q.out
index 50ae7e3..692ed66 100644
--- ql/src/test/results/clientpositive/union24.q.out
+++ ql/src/test/results/clientpositive/union24.q.out
@@ -1051,7 +1051,6 @@ ABSTRACT SYNTAX TREE:
STAGE DEPENDENCIES:
Stage-1 is a root stage
Stage-2 depends on stages: Stage-1
- Stage-3 depends on stages: Stage-2
Stage-0 is a root stage
STAGE PLANS:
@@ -1206,23 +1205,30 @@ STAGE PLANS:
keys:
expr: _col0
type: string
- mode: hash
+ mode: complete
outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
-#### A masked pattern was here ####
- NumFilesPerFileSink: 1
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- properties:
- columns _col0,_col1
- columns.types string,bigint
- escape.delim \
- TotalFiles: 1
- GatherStats: false
- MultiFileSpray: false
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ properties:
+ columns _col0,_col1
+ columns.types string,bigint
+ escape.delim \
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
Truncated Path -> Alias:
/src4 [null-subquery2:s-subquery2:a]
/src5 [null-subquery2:s-subquery2:b]
@@ -1231,77 +1237,6 @@ STAGE PLANS:
Map Reduce
Alias -> Map Operator Tree:
#### A masked pattern was here ####
- Reduce Output Operator
- key expressions:
- expr: _col0
- type: string
- sort order: +
- Map-reduce partition columns:
- expr: _col0
- type: string
- tag: -1
- value expressions:
- expr: _col1
- type: bigint
- Needs Tagging: false
- Path -> Alias:
-#### A masked pattern was here ####
- Path -> Partition:
-#### A masked pattern was here ####
- Partition
- base file name: -mr-10002
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- properties:
- columns _col0,_col1
- columns.types string,bigint
- escape.delim \
-
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- properties:
- columns _col0,_col1
- columns.types string,bigint
- escape.delim \
- Reduce Operator Tree:
- Group By Operator
- aggregations:
- expr: count(VALUE._col0)
- bucketGroup: false
- keys:
- expr: KEY._col0
- type: string
- mode: mergepartial
- outputColumnNames: _col0, _col1
- Select Operator
- expressions:
- expr: _col0
- type: string
- expr: _col1
- type: bigint
- outputColumnNames: _col0, _col1
- File Output Operator
- compressed: false
- GlobalTableId: 0
-#### A masked pattern was here ####
- NumFilesPerFileSink: 1
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- properties:
- columns _col0,_col1
- columns.types string,bigint
- escape.delim \
- TotalFiles: 1
- GatherStats: false
- MultiFileSpray: false
- Truncated Path -> Alias:
-#### A masked pattern was here ####
-
- Stage: Stage-3
- Map Reduce
- Alias -> Map Operator Tree:
-#### A masked pattern was here ####
TableScan
GatherStats: false
Union
@@ -1405,7 +1340,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: -mr-10003
+ base file name: -mr-10002
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
properties: