diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnTracker.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnTracker.java new file mode 100644 index 0000000..dedb91b --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnTracker.java @@ -0,0 +1,231 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import org.apache.hadoop.hive.ql.plan.*; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class ColumnTracker { + + private Operator current; + + private ColumnTracker(Operator current) { + this.current = current; + } + + public String track(String input) { + return input + "=" + resolveString(current, input); + } + + public String track(ExprNodeDesc expr) { + return expr.getExprString() + "=" + resolveString(current, expr); + } + + public static ColumnTracker create(Operator current) { + return new ColumnTracker(current); + } + + public static String resolveString(Operator op, String input) { + String[] resolved = resolve(op, new String[]{null, input}, -1); + return toString(resolved); + } + + public static String resolveString(Operator op, ExprNodeDesc expr) { + return resolveString(op, expr, -1); + } + + private static String resolveString(Operator op, ExprNodeDesc expr, int index) { + String[] resolved = resolve(op, expr, index); + return toString(resolved); + } + + private static String[] resolve(Operator op, String[] column, int index) { + return forward(op, column, index); + } + + private static String[] resolve(Operator op, ExprNodeDesc expr, int index) { + if (expr instanceof ExprNodeColumnDesc) { + ExprNodeColumnDesc column = (ExprNodeColumnDesc) expr; + return forward(op, column, index); + } + if (expr instanceof ExprNodeGenericFuncDesc) { + ExprNodeGenericFuncDesc func = (ExprNodeGenericFuncDesc) expr; + index = index < 0 ? indexFor(op, expr) : index; + List args = new ArrayList(); + for (ExprNodeDesc child : func.getChildExprs()) { + args.add(resolveString(op, child, index)); + } + String[] array = args.toArray(new String[args.size()]); + return new String[]{null, func.getGenericUDF().getDisplayString(array)}; + } + return toString(expr); + } + + private static String[] forward(Operator op, ExprNodeColumnDesc column, int index) { + index = index < 0 ? indexFor(op, column) : index; + Operator parent = parentFor(op, index); + if (parent != null) { + ExprNodeDesc resolved = backtracking(parent, column.getColumn()); + return resolve(parent, resolved != null ? resolved : column, -1); + } + return toString(column); + } + + private static String[] forward(Operator op, String[] column, int index) { + index = index < 0 ? indexFor(op, column) : index; + Operator parent = parentFor(op, index); + if (parent != null) { + ExprNodeDesc resolved = backtracking(parent, column[1]); + return resolved != null ? resolve(parent, resolved, -1) : resolve(parent, column, -1); + } + return column; + } + + private static String[] toString(ExprNodeDesc expr) { + if (expr instanceof ExprNodeColumnDesc) { + ExprNodeColumnDesc column = (ExprNodeColumnDesc) expr; + return new String[]{column.getTabAlias(), column.getColumn()}; + } + return new String[]{null, expr.getExprString()}; + } + + @SuppressWarnings("unchecked") + private static int indexFor(Operator op, ExprNodeDesc expr) { + if (op instanceof CommonJoinOperator) { + JoinDesc join = ((CommonJoinOperator) op).getConf(); + return indexFor(expr, join.getKeys(), join.getExprs()); + } + if (op instanceof HashTableSinkOperator) { + HashTableSinkDesc hsync = ((HashTableSinkOperator) op).getConf(); + return indexFor(expr, hsync.getKeys(), hsync.getExprs()); + } + return -1; + } + + @SuppressWarnings("unchecked") + private static int indexFor(Operator op, String[] column) { + if (op instanceof CommonJoinOperator) { + JoinDesc join = ((CommonJoinOperator) op).getConf(); + return indexFor(column, join.getKeys(), join.getExprs()); + } + if (op instanceof HashTableSinkOperator) { + HashTableSinkDesc hsync = ((HashTableSinkOperator) op).getConf(); + return indexFor(column, hsync.getKeys(), hsync.getExprs()); + } + return -1; + } + + private static int indexFor(ExprNodeDesc expr, Map>... mappings) { + for (Map> mapping : mappings) { + for (Map.Entry> entry : mapping.entrySet()) { + for (ExprNodeDesc mexpr : entry.getValue()) { + if (mexpr.isSame(expr) && isSameTable(expr, mexpr)) { + return entry.getKey(); + } + } + } + } + if (expr instanceof ExprNodeColumnDesc) { + throw new IllegalStateException("not found " + expr + " from " + Arrays.toString(mappings)); + } + return -1; + } + + private static boolean isSameTable(ExprNodeDesc expr, ExprNodeDesc mexpr) { + if (expr instanceof ExprNodeColumnDesc && mexpr instanceof ExprNodeColumnDesc) { + String alias1 = ((ExprNodeColumnDesc) expr).getTabAlias(); + String alias2 = ((ExprNodeColumnDesc) mexpr).getTabAlias(); + return (alias1 == null && alias2 == null) || (alias1 != null && alias1.equals(alias2)); + } + return true; + } + + private static int indexFor(String[] column, Map>... mappings) { + for (Map> mapping : mappings) { + for (Map.Entry> entry : mapping.entrySet()) { + for (ExprNodeDesc mexpr : entry.getValue()) { + if (isSame(column, mexpr)) { + return entry.getKey(); + } + } + } + } + throw new IllegalStateException("not found " + toString(column) + " from " + Arrays.toString(mappings)); + } + + private static boolean isSame(String[] column, ExprNodeDesc mexpr) { + if (mexpr instanceof ExprNodeColumnDesc) { + ExprNodeColumnDesc mcolumn = (ExprNodeColumnDesc) mexpr; + if (column[1].equals(mcolumn.getColumn()) && (column[0] == null || column[0].equals(mcolumn.getTabAlias()))) { + return true; + } + } + return false; + } + + private static String toString(String[] column) { + return (column[0] == null ? "" : column[0] + ".") + column[1]; + } + + // there should be exist a mapping expr for a column (should throw exception), + // but there is some cases like JOIN.key which should be evaluated from original RS not from JOIN + // In here, I just pass null, which makes forward to upper operator.. should be addressed later. + private static ExprNodeDesc backtracking(Operator op, String column) { + Map colMapper = op.getColumnExprMap(); + if (colMapper != null) { + return colMapper.get(column); + } + return null; + } + + private static Operator parentFor(Operator op, int index) { + if (op instanceof TableScanOperator && ((TableScanOperator) op).getSplitParent() != null) { + return ((TableScanOperator) op).getSplitParent(); + } + if (op.getParentOperators() == null || op.getParentOperators().isEmpty()) { + return null; + } + if (index >= 0) { + Operator target = op.getParentOperators().get(index); + if (target == null || target instanceof HashTableDummyOperator) { + target = peerFor(op).getParentOperators().get(index); + } + return target; + } + if (op.getParentOperators().size() > 1) { + throw new IllegalStateException("ambiguous parent for " + op); + } + return op.getParentOperators().get(0); + } + + private static Operator peerFor(Operator op) { + if (op instanceof MapJoinOperator) { + return ((MapJoinOperator) op).getPeer(); + } + if (op instanceof HashTableSinkOperator) { + return ((HashTableSinkOperator) op).getPeer(); + } + return null; + } +} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java index fe95cb1..dcef130 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java @@ -23,13 +23,7 @@ import java.io.PrintStream; import java.io.Serializable; import java.lang.annotation.Annotation; import java.lang.reflect.Method; -import java.util.Arrays; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeMap; +import java.util.*; import java.util.Map.Entry; import org.apache.hadoop.fs.Path; @@ -37,6 +31,7 @@ import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.plan.Explain; import org.apache.hadoop.hive.ql.plan.ExplainWork; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.util.StringUtils; @@ -94,7 +89,7 @@ public class ExplainTask extends Task implements Serializable { } private void outputMap(Map mp, String header, PrintStream out, - boolean extended, int indent) throws Exception { + boolean extended, Operator current, int indent) throws Exception { boolean first_el = true; TreeMap tree = new TreeMap(); @@ -118,7 +113,7 @@ public class ExplainTask extends Task implements Serializable { out.println(); } else if (ent.getValue() instanceof Serializable) { out.println(); - outputPlan((Serializable) ent.getValue(), out, extended, indent + 2); + outputPlan((Serializable) ent.getValue(), out, extended, current, indent + 2); } else { out.println(); } @@ -126,7 +121,7 @@ public class ExplainTask extends Task implements Serializable { } private void outputList(List l, String header, PrintStream out, - boolean extended, int indent) throws Exception { + boolean extended, Operator current, int indent) throws Exception { boolean first_el = true; boolean nl = false; @@ -148,7 +143,7 @@ public class ExplainTask extends Task implements Serializable { if (first_el) { out.println(); } - outputPlan((Serializable) o, out, extended, indent + 2); + outputPlan((Serializable) o, out, extended, current, indent + 2); } first_el = false; @@ -174,7 +169,7 @@ public class ExplainTask extends Task implements Serializable { } private void outputPlan(Serializable work, PrintStream out, boolean extended, - int indent) throws Exception { + Operator current, int indent) throws Exception { // Check if work has an explain annotation Annotation note = work.getClass().getAnnotation(Explain.class); @@ -192,11 +187,11 @@ public class ExplainTask extends Task implements Serializable { if (work instanceof Operator) { Operator operator = (Operator) work; if (operator.getConf() != null) { - outputPlan(operator.getConf(), out, extended, indent); + outputPlan(operator.getConf(), out, extended, operator, indent); } if (operator.getChildOperators() != null) { for (Operator op : operator.getChildOperators()) { - outputPlan(op, out, extended, indent + 2); + outputPlan(op, out, extended, op, indent + 2); } } return; @@ -220,6 +215,9 @@ public class ExplainTask extends Task implements Serializable { if (val == null) { continue; } + if (xpl_note.columnTrack()) { + val = trackColumns(current, val); + } String header = null; if (!xpl_note.displayName().equals("")) { @@ -239,7 +237,7 @@ public class ExplainTask extends Task implements Serializable { try { // Go through the map and print out the stuff Map mp = (Map) val; - outputMap(mp, header, out, extended, prop_indents + 2); + outputMap(mp, header, out, extended, current, prop_indents + 2); continue; } catch (ClassCastException ce) { // Ignore - all this means is that this is not a map @@ -248,7 +246,7 @@ public class ExplainTask extends Task implements Serializable { // Try this as a list try { List l = (List) val; - outputList(l, header, out, extended, prop_indents + 2); + outputList(l, header, out, extended, current, prop_indents + 2); continue; } catch (ClassCastException ce) { @@ -259,7 +257,7 @@ public class ExplainTask extends Task implements Serializable { try { Serializable s = (Serializable) val; out.println(header); - outputPlan(s, out, extended, prop_indents + 2); + outputPlan(s, out, extended, current, prop_indents + 2); continue; } catch (ClassCastException ce) { @@ -270,6 +268,39 @@ public class ExplainTask extends Task implements Serializable { } } + private Object trackColumns(Operator current, Object val) { + if (val instanceof List) { + List details = new ArrayList(); + for (Object column : (List)val) { + details.add(trackColumns(current, column)); + } + return details; + } + if (val instanceof Map) { + Mapresult = new HashMap(); + for (Map.Entry entry : ((Map)val).entrySet()) { + result.put(entry.getKey(), trackColumns(current, entry.getValue())); + } + return result; + } + return trackColumn(current, val); + } + + private String trackColumn(Operator current, Object val) { + if (val instanceof String) { + String column = (String) val; + String resolved = ColumnTracker.resolveString(current, column); + return column.equals(resolved) ? column : column + "=" + resolved; + } + if (val instanceof ExprNodeDesc) { + ExprNodeDesc expr = (ExprNodeDesc) val; + String original = expr.getExprString(); + String resolved = ColumnTracker.resolveString(current, expr); + return original.equals(resolved) ? original : original + "=" + resolved; + } + return String.valueOf(val); + } + private void outputPlan(Task task, PrintStream out, boolean extended, HashSet> displayedSet, int indent) throws Exception { @@ -283,7 +314,7 @@ public class ExplainTask extends Task implements Serializable { out.printf("Stage: %s\n", task.getId()); // Start by getting the work part of the task and call the output plan for // the work - outputPlan(task.getWork(), out, extended, indent + 2); + outputPlan(task.getWork(), out, extended, null, indent + 2); out.println(); if (task instanceof ConditionalTask && ((ConditionalTask) task).getListTasks() != null) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java index 9cffc10..9eea29f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java @@ -118,6 +118,8 @@ public class HashTableSinkOperator extends TerminalOperator i private long hashTableScale; private boolean isAbort = false; + private transient Operator peer; + public static class HashTableSinkObjectCtx { ObjectInspector standardOI; SerDe serde; @@ -443,4 +445,12 @@ public class HashTableSinkOperator extends TerminalOperator i public OperatorType getType() { return OperatorType.HASHTABLESINK; } + + public Operator getPeer() { + return peer; + } + + public void setPeer(Operator peer) { + this.peer = peer; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java index c7784c9..52284a5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java @@ -67,6 +67,8 @@ public class MapJoinOperator extends AbstractMapJoinOperator implem transient boolean hashTblInitedOnce; private int bigTableAlias; + private transient Operator peer; + public MapJoinOperator() { } @@ -153,7 +155,7 @@ public class MapJoinOperator extends AbstractMapJoinOperator implem hashTblInitedOnce = true; } } - + boolean localMode = HiveConf.getVar(hconf, HiveConf.ConfVars.HADOOPJT).equals("local"); String baseDir = null; @@ -301,7 +303,7 @@ public class MapJoinOperator extends AbstractMapJoinOperator implem public void closeOp(boolean abort) throws HiveException { if (mapJoinTables != null) { - for (HashMapWrapper hashTable : mapJoinTables.values()) { + for (HashMapWrapper hashTable : mapJoinTables.values()) { hashTable.close(); } } @@ -322,4 +324,12 @@ public class MapJoinOperator extends AbstractMapJoinOperator implem public OperatorType getType() { return OperatorType.MAPJOIN; } + + public Operator getPeer() { + return peer; + } + + public void setPeer(Operator peer) { + this.peer = peer; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java index 141df13..8f91658 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java @@ -60,6 +60,8 @@ public class TableScanOperator extends Operator implements private transient Stat currentStat; private transient Map stats; + private Operator splitParent; + public TableDesc getTableDesc() { return tableDesc; } @@ -262,4 +264,12 @@ public class TableScanOperator extends Operator implements } statsPublisher.closeConnection(); } + + public Operator getSplitParent() { + return splitParent; + } + + public void setSplitParent(Operator parent) { + this.splitParent = parent; + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java index 6d9b988..52d63f8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java @@ -468,8 +468,8 @@ public final class ColumnPrunerProcFactory { private static void pruneReduceSinkOperator(boolean[] retainFlags, ReduceSinkOperator reduce, ColumnPrunerProcCtx cppCtx) throws SemanticException { ReduceSinkDesc reduceConf = reduce.getConf(); + Map oldMap = reduce.getColumnExprMap(); - Map newMap = new HashMap(); ArrayList sig = new ArrayList(); RowResolver oldRR = cppCtx.getOpToParseCtxMap().get(reduce).getRowResolver(); RowResolver newRR = new RowResolver(); @@ -489,10 +489,11 @@ public final class ColumnPrunerProcFactory { outputCol = Utilities.ReduceField.VALUE.toString() + "." + outputCol; nm = oldRR.reverseLookup(outputCol); } - newMap.put(outputCol, oldMap.get(outputCol)); ColumnInfo colInfo = oldRR.get(nm[0], nm[1]); newRR.put(nm[0], nm[1], colInfo); sig.add(colInfo); + } else { + oldMap.remove(originalValueOutputColNames.get(i)); } } @@ -513,7 +514,6 @@ public final class ColumnPrunerProcFactory { } cppCtx.getOpToParseCtxMap().get(reduce).setRowResolver(newRR); - reduce.setColumnExprMap(newMap); reduce.getSchema().setSignature(sig); reduceConf.setOutputValueColumnNames(newOutputColNames); reduceConf.setValueCols(newValueEval); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java index ae33079..0fd04fc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java @@ -952,11 +952,12 @@ public final class GenMapRedUtils { // create a dummy tableScan operator on top of op // TableScanOperator is implicitly created here for each MapOperator RowResolver rowResolver = opProcCtx.getParseCtx().getOpParseCtx().get(parent).getRowResolver(); - Operator ts_op = putOpInsertMap(OperatorFactory + TableScanOperator ts_op = (TableScanOperator) putOpInsertMap(OperatorFactory .get(TableScanDesc.class, parent.getSchema()), rowResolver, parseCtx); childOpList = new ArrayList>(); childOpList.add(op); + ts_op.setSplitParent(parent); ts_op.setChildOperators(childOpList); op.getParentOperators().set(posn, ts_op); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java index 8f2fef4..6d26238 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java @@ -181,6 +181,8 @@ public final class LocalMapJoinProcFactory { for (Operator op : dummyOperators) { context.addDummyParentOp(op); } + hashTableSinkOp.setPeer(mapJoinOp); + mapJoinOp.setPeer(hashTableSinkOp); return null; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index dc9ab1b..a14b264 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -2439,6 +2439,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { field, udaf.returnType, "", false)); // Save the evaluator so that it can be used by the next-stage // GroupByOperators + colExprMap.put(field, ExprNodeGenericFuncDesc.newInstance(aggName, genericUDAFEvaluator, amode, aggParameters, isDistinct)); if (genericUDAFEvaluators != null) { genericUDAFEvaluators.put(entry.getKey(), genericUDAFEvaluator); } @@ -2588,6 +2589,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { (mode != GroupByDesc.Mode.FINAL && isDistinct), amode)); String field = getColumnInternalName(groupByKeys.size() + aggregations.size() - 1); + colExprMap.put(field, ExprNodeGenericFuncDesc.newInstance(aggName.toLowerCase(), + genericUDAFEvaluator, amode, aggParameters, isDistinct)); + outputColumnNames.add(field); groupByOutputRowResolver.putExpression(value, new ColumnInfo( field, udaf.returnType, "", false)); @@ -2702,6 +2706,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { amode)); String field = getColumnInternalName(groupByKeys.size() + aggregations.size() - 1); + colExprMap.put(field, ExprNodeGenericFuncDesc.newInstance(aggName.toLowerCase(), + genericUDAFEvaluator, amode, aggParameters, isDistinct)); outputColumnNames.add(field); groupByOutputRowResolver.putExpression(value, new ColumnInfo( field, udaf.returnType, "", false)); @@ -2805,6 +2811,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { + "." + name; ColumnInfo colInfo = new ColumnInfo(field, expr.getTypeInfo(), null, false); reduceSinkOutputRowResolver.putExpression(parameter, colInfo); + colExprMap.put(colInfo.getInternalName(), expr); numExprs++; } distinctColIndices.add(distinctIndices); @@ -2829,6 +2836,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { .add(getColumnInternalName(reduceValues.size() - 1)); String field = Utilities.ReduceField.VALUE.toString() + "." + getColumnInternalName(reduceValues.size() - 1); + colExprMap.put(field, reduceValues.get(reduceValues.size() - 1)); reduceSinkOutputRowResolver.putExpression(parameter, new ColumnInfo(field, reduceValues.get(reduceValues.size() - 1).getTypeInfo(), null, false)); @@ -2849,6 +2857,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { outputValueColumnNames.add(getColumnInternalName(reduceValues.size() - 1)); String field = Utilities.ReduceField.VALUE.toString() + "." + getColumnInternalName(reduceValues.size() - 1); + colExprMap.put(field, reduceValues.get(reduceValues.size() - 1)); + reduceSinkOutputRowResolver.putExpression(entry.getValue(), new ColumnInfo(field, type, null, false)); } @@ -2924,6 +2934,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { reduceSinkOutputRowResolver2.putExpression(t, new ColumnInfo( Utilities.ReduceField.VALUE.toString() + "." + col, typeInfo, "", false)); + colExprMap.put(Utilities.ReduceField.VALUE.toString() + "." + col, + reduceValues.get(reduceValues.size() - 1)); } ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap( @@ -3016,6 +3028,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { String field = getColumnInternalName(groupByKeys.size() + aggregations.size() - 1); outputColumnNames.add(field); + colExprMap.put(field, ExprNodeGenericFuncDesc.newInstance(aggName.toLowerCase(), + genericUDAFEvaluator, amode, aggParameters, isDistinct)); groupByOutputRowResolver2.putExpression(value, new ColumnInfo( field, udaf.returnType, "", false)); } @@ -4579,13 +4593,20 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { rightOps[pos] = input; } + Map> keyExprMap = new HashMap>(); + for (int pos = 0; pos < rightOps.length; pos++) { + ReduceSinkOperator rs = (ReduceSinkOperator) rightOps[pos]; + ReduceSinkDesc rsconf = rs.getConf(); + keyExprMap.put((byte) rsconf.getTag(), rsconf.getKeyCols()); + } + JoinCondDesc[] joinCondns = new JoinCondDesc[join.getJoinCond().length]; for (int i = 0; i < join.getJoinCond().length; i++) { JoinCond condn = join.getJoinCond()[i]; joinCondns[i] = new JoinCondDesc(condn); } - JoinDesc desc = new JoinDesc(exprMap, outputColumnNames, + JoinDesc desc = new JoinDesc(keyExprMap, exprMap, outputColumnNames, join.getNoOuterJoin(), joinCondns, filterMap); desc.setReversedExprs(reversedExprs); JoinOperator joinOp = (JoinOperator) OperatorFactory.getAndMakeChild(desc, @@ -4602,18 +4623,20 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { RowResolver outputRS = new RowResolver(); ArrayList outputColumns = new ArrayList(); ArrayList reduceKeys = new ArrayList(); + Map colExprMap = new HashMap(); // Compute join keys and store in reduceKeys ArrayList exprs = joinTree.getExpressions().get(pos); for (int i = 0; i < exprs.size(); i++) { ASTNode expr = exprs.get(i); reduceKeys.add(genExprNodeDesc(expr, inputRS)); + String field = Utilities.ReduceField.KEY.toString() + "." + getColumnInternalName(i); + colExprMap.put(field, reduceKeys.get(i)); } // Walk over the input row resolver and copy in the output ArrayList reduceValues = new ArrayList(); Iterator tblNamesIter = inputRS.getTableNames().iterator(); - Map colExprMap = new HashMap(); while (tblNamesIter.hasNext()) { String src = tblNamesIter.next(); HashMap fMap = inputRS.getFieldMap(src); @@ -5355,7 +5378,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( new SelectDesc(colList, columnNames, true), new RowSchema(inputRR .getColumnInfos()), input), inputRR); - output.setColumnExprMap(input.getColumnExprMap()); return output; } @@ -5475,6 +5497,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { reduceValues.size() - 1).getTypeInfo(), "", false); reduceSinkOutputRowResolver.putExpression(grpbyExpr, colInfo); outputColumnNames.add(getColumnInternalName(reduceValues.size() - 1)); + colExprMap.put(field, grpByExprNode); } } @@ -5501,6 +5524,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { reduceSinkOutputRowResolver.putExpression(paraExpr, colInfo); outputColumnNames .add(getColumnInternalName(reduceValues.size() - 1)); + colExprMap.put(field, paraExprNode); } } } @@ -5677,6 +5701,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer { reduceSinkOutputRowResolver.putExpression(paraExpr, colInfo); outputColumnNames .add(getColumnInternalName(reduceValues.size() - 1)); + colExprMap.put(colInfo.getInternalName(), paraExprNode); } } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AggregationDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AggregationDesc.java index ab70bab..e169e32 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AggregationDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AggregationDesc.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.plan; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; /** @@ -95,24 +96,8 @@ public class AggregationDesc implements java.io.Serializable { return mode; } - @Explain(displayName = "expr") - public String getExprString() { - StringBuilder sb = new StringBuilder(); - sb.append(genericUDAFName); - sb.append("("); - if (distinct) { - sb.append("DISTINCT "); - } - boolean first = true; - for (ExprNodeDesc exp : parameters) { - if (first) { - first = false; - } else { - sb.append(", "); - } - sb.append(exp.getExprString()); - } - sb.append(")"); - return sb.toString(); + @Explain(displayName = "expr", columnTrack = true) + public ExprNodeDesc getExprString() throws UDFArgumentException { + return ExprNodeGenericFuncDesc.newInstance(genericUDAFName, genericUDAFEvaluator, mode, parameters, distinct); } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/Explain.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/Explain.java index 85c9f8f..b434a4a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/Explain.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/Explain.java @@ -30,4 +30,6 @@ public @interface Explain { String displayName() default ""; boolean normalExplain() default true; + + boolean columnTrack() default false; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java index e264709..f96c0fc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeGenericFuncDesc.java @@ -25,6 +25,9 @@ import java.util.List; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFWrapper; import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -155,6 +158,24 @@ public class ExprNodeGenericFuncDesc extends ExprNodeDesc implements .getTypeInfoFromObjectInspector(oi), genericUDF, children); } + public static ExprNodeGenericFuncDesc newInstance(String name, GenericUDAFEvaluator genericUDAF, + GenericUDAFEvaluator.Mode m, List children, boolean distinct) throws UDFArgumentException { + ObjectInspector[] childrenOIs = new ObjectInspector[children.size()]; + for (int i = 0; i < childrenOIs.length; i++) { + childrenOIs[i] = TypeInfoUtils + .getStandardWritableObjectInspectorFromTypeInfo(children.get(i) + .getTypeInfo()); + } + + try { + ObjectInspector oi = genericUDAF.init(m, childrenOIs); + return new ExprNodeGenericFuncDesc(TypeInfoUtils + .getTypeInfoFromObjectInspector(oi), new GenericUDAFWrapper(name, distinct), children); + } catch (HiveException e) { + throw new UDFArgumentException(e); + } + } + @Override public boolean isSame(Object o) { if (!(o instanceof ExprNodeGenericFuncDesc)) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FilterDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/FilterDesc.java index 769c70d..feb2582 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FilterDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FilterDesc.java @@ -88,7 +88,7 @@ public class FilterDesc implements Serializable { this.sampleDescr = sampleDescr; } - @Explain(displayName = "predicate") + @Explain(displayName = "predicate", columnTrack = true) public org.apache.hadoop.hive.ql.plan.ExprNodeDesc getPredicate() { return predicate; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/GroupByDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/GroupByDesc.java index 9ebbb33..0d510e6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/GroupByDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/GroupByDesc.java @@ -114,7 +114,7 @@ public class GroupByDesc implements java.io.Serializable { this.mode = mode; } - @Explain(displayName = "keys") + @Explain(displayName = "keys", columnTrack = true) public java.util.ArrayList getKeys() { return keys; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java index 4a729e8..29742e0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HashTableSinkDesc.java @@ -284,7 +284,7 @@ public class HashTableSinkDesc extends JoinDesc implements Serializable { /** * @return the keys */ - @Explain(displayName = "keys") + @Explain(displayName = "keys", columnTrack = true) public Map> getKeys() { return keys; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java index df9097d..6f5013c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java @@ -46,6 +46,9 @@ public class JoinDesc implements Serializable { private Map skewKeysValuesTables; // alias to key mapping + private Map> keys; + + // alias to value mapping private Map> exprs; // alias to filter mapping @@ -68,9 +71,10 @@ public class JoinDesc implements Serializable { public JoinDesc() { } - public JoinDesc(final Map> exprs, + public JoinDesc(Map> keys, final Map> exprs, List outputColumnNames, final boolean noOuterJoin, final JoinCondDesc[] conds, final Map> filters) { + this.keys = keys; this.exprs = exprs; this.outputColumnNames = outputColumnNames; this.noOuterJoin = noOuterJoin; @@ -83,23 +87,24 @@ public class JoinDesc implements Serializable { } } - public JoinDesc(final Map> exprs, + public JoinDesc(Map> keys, final Map> exprs, List outputColumnNames, final boolean noOuterJoin, final JoinCondDesc[] conds) { - this(exprs, outputColumnNames, noOuterJoin, conds, null); + this(keys, exprs, outputColumnNames, noOuterJoin, conds, null); } - public JoinDesc(final Map> exprs, + public JoinDesc(Map> keys, final Map> exprs, List outputColumnNames) { - this(exprs, outputColumnNames, true, null); + this(keys, exprs, outputColumnNames, true, null); } - public JoinDesc(final Map> exprs, + public JoinDesc(Map> keys, final Map> exprs, List outputColumnNames, final JoinCondDesc[] conds) { - this(exprs, outputColumnNames, true, conds, null); + this(keys, exprs, outputColumnNames, true, conds, null); } public JoinDesc(JoinDesc clone) { + this.keys = clone.keys; this.bigKeysDirMap = clone.bigKeysDirMap; this.conds = clone.conds; this.exprs = clone.exprs; @@ -115,6 +120,7 @@ public class JoinDesc implements Serializable { this.filters = clone.filters; } + @Explain(displayName = "condition expressions", columnTrack = true) public Map> getExprs() { return exprs; } @@ -127,33 +133,13 @@ public class JoinDesc implements Serializable { this.reversedExprs = reversedExprs; } - @Explain(displayName = "condition expressions") - public Map getExprsStringMap() { - if (getExprs() == null) { - return null; - } - - LinkedHashMap ret = new LinkedHashMap(); - - for (Map.Entry> ent : getExprs().entrySet()) { - StringBuilder sb = new StringBuilder(); - boolean first = true; - if (ent.getValue() != null) { - for (ExprNodeDesc expr : ent.getValue()) { - if (!first) { - sb.append(" "); - } - - first = false; - sb.append("{"); - sb.append(expr.getExprString()); - sb.append("}"); - } - } - ret.put(ent.getKey(), sb.toString()); - } + @Explain(displayName = "keys", columnTrack = true) + public Map> getKeys() { + return keys; + } - return ret; + public void setKeys(Map> keys) { + this.keys = keys; } public void setExprs(final Map> exprs) { @@ -167,41 +153,9 @@ public class JoinDesc implements Serializable { * * @return Map from alias to filters on the alias. */ - @Explain(displayName = "filter predicates") - public Map getFiltersStringMap() { - if (getFilters() == null || getFilters().size() == 0) { - return null; - } - - LinkedHashMap ret = new LinkedHashMap(); - boolean filtersPresent = false; - - for (Map.Entry> ent : getFilters().entrySet()) { - StringBuilder sb = new StringBuilder(); - boolean first = true; - if (ent.getValue() != null) { - if (ent.getValue().size() != 0) { - filtersPresent = true; - } - for (ExprNodeDesc expr : ent.getValue()) { - if (!first) { - sb.append(" "); - } - - first = false; - sb.append("{"); - sb.append(expr.getExprString()); - sb.append("}"); - } - } - ret.put(ent.getKey(), sb.toString()); - } - - if (filtersPresent) { - return ret; - } else { - return null; - } + @Explain(displayName = "filter predicates", columnTrack = true) + public Map> getFiltersStringMap() { + return filters; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java index ada9826..59a7709 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java @@ -36,7 +36,6 @@ import java.util.Map.Entry; public class MapJoinDesc extends JoinDesc implements Serializable { private static final long serialVersionUID = 1L; - private Map> keys; private TableDesc keyTblDesc; private List valueTblDescs; private List valueFilteredTblDescs; @@ -56,7 +55,6 @@ public class MapJoinDesc extends JoinDesc implements Serializable { public MapJoinDesc(MapJoinDesc clone) { super(clone); - this.keys = clone.keys; this.keyTblDesc = clone.keyTblDesc; this.valueTblDescs = clone.valueTblDescs; this.posBigTable = clone.posBigTable; @@ -71,8 +69,7 @@ public class MapJoinDesc extends JoinDesc implements Serializable { final List valueTblDescs,final List valueFilteredTblDescs, List outputColumnNames, final int posBigTable, final JoinCondDesc[] conds, final Map> filters, boolean noOuterJoin) { - super(values, outputColumnNames, noOuterJoin, conds, filters); - this.keys = keys; + super(keys, values, outputColumnNames, noOuterJoin, conds, filters); this.keyTblDesc = keyTblDesc; this.valueTblDescs = valueTblDescs; this.valueFilteredTblDescs = valueFilteredTblDescs; @@ -104,22 +101,6 @@ public class MapJoinDesc extends JoinDesc implements Serializable { } /** - * @return the keys - */ - @Explain(displayName = "keys") - public Map> getKeys() { - return keys; - } - - /** - * @param keys - * the keys to set - */ - public void setKeys(Map> keys) { - this.keys = keys; - } - - /** * @return the position of the big table not in memory */ @Explain(displayName = "Position of Big Table") diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java index d7f0476..a62afef 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java @@ -109,7 +109,7 @@ public class ReduceSinkDesc implements Serializable { this.outputValueColumnNames = outputValueColumnNames; } - @Explain(displayName = "key expressions") + @Explain(displayName = "key expressions", columnTrack = true) public java.util.ArrayList getKeyCols() { return keyCols; } @@ -126,7 +126,7 @@ public class ReduceSinkDesc implements Serializable { this.numDistributionKeys = numKeys; } - @Explain(displayName = "value expressions") + @Explain(displayName = "value expressions", columnTrack = true) public java.util.ArrayList getValueCols() { return valueCols; } @@ -135,7 +135,7 @@ public class ReduceSinkDesc implements Serializable { this.valueCols = valueCols; } - @Explain(displayName = "Map-reduce partition columns") + @Explain(displayName = "Map-reduce partition columns", columnTrack = true) public java.util.ArrayList getPartitionCols() { return partitionCols; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/SelectDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/SelectDesc.java index 97ae10d..09ff874 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/SelectDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/SelectDesc.java @@ -62,7 +62,7 @@ public class SelectDesc implements Serializable { this.selStarNoCompute = selStarNoCompute; } - @Explain(displayName = "expressions") + @Explain(displayName = "expressions", columnTrack = true) public java.util.ArrayList getColList() { return colList; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFWrapper.java new file mode 100644 index 0000000..e143fc8 --- /dev/null +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFWrapper.java @@ -0,0 +1,61 @@ +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; + +public class GenericUDAFWrapper extends GenericUDF { + + private boolean distinct; + private String funcName; + + public GenericUDAFWrapper() {} + + public GenericUDAFWrapper(String funcName, boolean distinct) { + this.funcName = funcName; + this.distinct = distinct; + } + + public boolean getDistinct() { + return distinct; + } + + public void setDistinct(boolean distinct) { + this.distinct = distinct; + } + + public String getGenericUDAF() { + return funcName; + } + + public void setGenericUDAF(String funcName) { + this.funcName = funcName; + } + + @Override + public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException { + throw new IllegalStateException(); + } + + @Override + public Object evaluate(DeferredObject[] arguments) throws HiveException { + throw new IllegalStateException(); + } + + @Override + public String getDisplayString(String[] children) { + StringBuilder sb = new StringBuilder(); + sb.append(funcName).append('('); + if (distinct) { + sb.append("DISTINCT "); + } + if (children.length > 0) { + for (int i = 0; i < children.length - 1; i++) { + sb.append(children[i]).append(", "); + } + sb.append(children[children.length - 1]); + } + sb.append(')'); + return sb.toString(); + } +} diff --git a/ql/src/test/queries/clientpositive/explain_columns.q b/ql/src/test/queries/clientpositive/explain_columns.q new file mode 100644 index 0000000..9bb1003 --- /dev/null +++ b/ql/src/test/queries/clientpositive/explain_columns.q @@ -0,0 +1,67 @@ +create table t1( key_int1 int, key_int2 int, key_string1 string, key_string2 string); +create table t2( key_int1 int, key_int2 int, key_string1 string, key_string2 string); +create table t3( key_int1 int, key_int2 int, key_string1 string, key_string2 string); + +set hive.map.aggr = false; +set hive.auto.convert.join = false; + +explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1; + +set hive.map.aggr = false; +set hive.auto.convert.join = true; + +explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1; + +set hive.map.aggr = true; +set hive.auto.convert.join = false; + +explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1; + +set hive.map.aggr = true; +set hive.auto.convert.join = true; + +explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1; + +set hive.map.aggr = false; +set hive.auto.convert.join = false; + +explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1); + +set hive.map.aggr = false; +set hive.auto.convert.join = true; + +explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1); + +set hive.map.aggr = true; +set hive.auto.convert.join = false; + +explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1); + +set hive.map.aggr = true; +set hive.auto.convert.join = true; + +explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1); diff --git a/ql/src/test/results/clientpositive/explain_columns.q.out b/ql/src/test/results/clientpositive/explain_columns.q.out new file mode 100644 index 0000000..3ce3baf --- /dev/null +++ b/ql/src/test/results/clientpositive/explain_columns.q.out @@ -0,0 +1,1657 @@ +PREHOOK: query: create table t1( key_int1 int, key_int2 int, key_string1 string, key_string2 string) +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table t1( key_int1 int, key_int2 int, key_string1 string, key_string2 string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@t1 +PREHOOK: query: create table t2( key_int1 int, key_int2 int, key_string1 string, key_string2 string) +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table t2( key_int1 int, key_int2 int, key_string1 string, key_string2 string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@t2 +PREHOOK: query: create table t3( key_int1 int, key_int2 int, key_string1 string, key_string2 string) +PREHOOK: type: CREATETABLE +POSTHOOK: query: create table t3( key_int1 int, key_int2 int, key_string1 string, key_string2 string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: default@t3 +PREHOOK: query: explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1 +PREHOOK: type: QUERY +POSTHOOK: query: explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME t2)) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME t1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key_int1)) (TOK_SELEXPR (TOK_TABLE_OR_COL key_string1)) (TOK_SELEXPR (TOK_FUNCTION upper (TOK_TABLE_OR_COL key_string2)) ref)))) Q1) (= (. (TOK_TABLE_OR_COL Q1) ref) (. (TOK_TABLE_OR_COL t2) key_string1))) (TOK_TABREF (TOK_TABNAME t3)) (= (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL Q1) ref) (. (TOK_TABLE_OR_COL t2) key_string1)) (. (TOK_TABLE_OR_COL t3) key_string1)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECTDI (TOK_SELEXPR (. (TOK_TABLE_OR_COL Q1) ref)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL t2) key_int1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL t3) key_string1))))) + +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-1 depends on stages: Stage-3 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + q1:t1 + TableScan + alias: t1 + Select Operator + expressions: upper(key_string2)=upper(t1.key_string2) + outputColumnNames: _col2 + Reduce Output Operator + key expressions: _col2=upper(t1.key_string2) + sort order: + + Map-reduce partition columns: _col2=upper(t1.key_string2) + tag: 1 + value expressions: _col2=upper(t1.key_string2) + t2 + TableScan + alias: t2 + Reduce Output Operator + key expressions: key_string1=t2.key_string1 + sort order: + + Map-reduce partition columns: key_string1=t2.key_string1 + tag: 0 + value expressions: key_int1=t2.key_int1, key_string1=t2.key_string1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t2.key_int1, VALUE._col2=t2.key_string1] + 1 [VALUE._col2=upper(t1.key_string2)] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + outputColumnNames: _col0, _col2, _col8 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1) + sort order: + + Map-reduce partition columns: concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1) + tag: 0 + value expressions: _col0=t2.key_int1, _col8=upper(t1.key_string2) + t3 + TableScan + alias: t3 + Reduce Output Operator + key expressions: key_string1=t3.key_string1 + sort order: + + Map-reduce partition columns: key_string1=t3.key_string1 + tag: 1 + value expressions: key_string1=t3.key_string1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t2.key_int1, VALUE._col8=upper(t1.key_string2)] + 1 [VALUE._col2=t3.key_string1] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + outputColumnNames: _col0, _col8, _col11 + Select Operator + expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + outputColumnNames: _col8, _col0, _col11 + 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: + file:/tmp/navis/hive_2011-08-05_00-39-39_110_6217657599011317634/-mr-10002 + Reduce Output Operator + key expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + sort order: +++ + Map-reduce partition columns: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + tag: -1 + Reduce Operator Tree: + Group By Operator + bucketGroup: false + keys: KEY._col0=upper(t1.key_string2), KEY._col1=t2.key_int1, KEY._col2=t3.key_string1 + mode: complete + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0=upper(t1.key_string2), _col1=t2.key_int1, _col2=t3.key_string1 + 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 distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1 +PREHOOK: type: QUERY +POSTHOOK: query: explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME t2)) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME t1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key_int1)) (TOK_SELEXPR (TOK_TABLE_OR_COL key_string1)) (TOK_SELEXPR (TOK_FUNCTION upper (TOK_TABLE_OR_COL key_string2)) ref)))) Q1) (= (. (TOK_TABLE_OR_COL Q1) ref) (. (TOK_TABLE_OR_COL t2) key_string1))) (TOK_TABREF (TOK_TABNAME t3)) (= (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL Q1) ref) (. (TOK_TABLE_OR_COL t2) key_string1)) (. (TOK_TABLE_OR_COL t3) key_string1)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECTDI (TOK_SELEXPR (. (TOK_TABLE_OR_COL Q1) ref)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL t2) key_int1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL t3) key_string1))))) + +STAGE DEPENDENCIES: + Stage-10 is a root stage , consists of Stage-13, Stage-14, Stage-3 + Stage-13 has a backup stage: Stage-3 + Stage-8 depends on stages: Stage-13 + Stage-7 depends on stages: Stage-3, Stage-8, Stage-9 , consists of Stage-11, Stage-12, Stage-1 + Stage-11 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-11 + Stage-2 depends on stages: Stage-1, Stage-5, Stage-6 + Stage-12 has a backup stage: Stage-1 + Stage-6 depends on stages: Stage-12 + Stage-1 + Stage-14 has a backup stage: Stage-3 + Stage-9 depends on stages: Stage-14 + Stage-3 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-10 + Conditional Operator + + Stage: Stage-13 + Map Reduce Local Work + Alias -> Map Local Tables: + q1:t1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + q1:t1 + TableScan + alias: t1 + Select Operator + expressions: upper(key_string2)=upper(t1.key_string2) + outputColumnNames: _col2 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + Position of Big Table: 0 + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + t2 + TableScan + alias: t2 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [key_int1=t2.key_int1, key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + outputColumnNames: _col0, _col2, _col8 + Position of Big Table: 0 + 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-7 + Conditional Operator + + Stage: Stage-11 + Map Reduce Local Work + Alias -> Map Local Tables: + t3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + t3 + TableScan + alias: t3 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + Position of Big Table: 0 + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [_col0=t2.key_int1, _col8=upper(t1.key_string2)] + 1 [key_string1=t3.key_string1] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + outputColumnNames: _col0, _col8, _col11 + Position of Big Table: 0 + Select Operator + expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + outputColumnNames: _col8, _col0, _col11 + 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: + file:/tmp/navis/hive_2011-08-05_00-39-39_613_6832304784840687491/-mr-10002 + Reduce Output Operator + key expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + sort order: +++ + Map-reduce partition columns: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + tag: -1 + Reduce Operator Tree: + Group By Operator + bucketGroup: false + keys: KEY._col0=upper(t1.key_string2), KEY._col1=t2.key_int1, KEY._col2=t3.key_string1 + mode: complete + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0=upper(t1.key_string2), _col1=t2.key_int1, _col2=t3.key_string1 + 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-12 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + Position of Big Table: 1 + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: + t3 + TableScan + alias: t3 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [_col0=t2.key_int1, _col8=upper(t1.key_string2)] + 1 [key_string1=t3.key_string1] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + outputColumnNames: _col0, _col8, _col11 + Position of Big Table: 1 + Select Operator + expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + outputColumnNames: _col8, _col0, _col11 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1) + sort order: + + Map-reduce partition columns: concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1) + tag: 0 + value expressions: _col0=t2.key_int1, _col8=upper(t1.key_string2) + t3 + TableScan + alias: t3 + Reduce Output Operator + key expressions: key_string1=t3.key_string1 + sort order: + + Map-reduce partition columns: key_string1=t3.key_string1 + tag: 1 + value expressions: key_string1=t3.key_string1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t2.key_int1, VALUE._col8=upper(t1.key_string2)] + 1 [VALUE._col2=t3.key_string1] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + outputColumnNames: _col0, _col8, _col11 + Select Operator + expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + outputColumnNames: _col8, _col0, _col11 + 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-14 + Map Reduce Local Work + Alias -> Map Local Tables: + t2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + t2 + TableScan + alias: t2 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + Position of Big Table: 1 + + Stage: Stage-9 + Map Reduce + Alias -> Map Operator Tree: + q1:t1 + TableScan + alias: t1 + Select Operator + expressions: upper(key_string2)=upper(t1.key_string2) + outputColumnNames: _col2 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [key_int1=t2.key_int1, key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + outputColumnNames: _col0, _col2, _col8 + Position of Big Table: 1 + 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: + q1:t1 + TableScan + alias: t1 + Select Operator + expressions: upper(key_string2)=upper(t1.key_string2) + outputColumnNames: _col2 + Reduce Output Operator + key expressions: _col2=upper(t1.key_string2) + sort order: + + Map-reduce partition columns: _col2=upper(t1.key_string2) + tag: 1 + value expressions: _col2=upper(t1.key_string2) + t2 + TableScan + alias: t2 + Reduce Output Operator + key expressions: key_string1=t2.key_string1 + sort order: + + Map-reduce partition columns: key_string1=t2.key_string1 + tag: 0 + value expressions: key_int1=t2.key_int1, key_string1=t2.key_string1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t2.key_int1, VALUE._col2=t2.key_string1] + 1 [VALUE._col2=upper(t1.key_string2)] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + outputColumnNames: _col0, _col2, _col8 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1 +PREHOOK: type: QUERY +POSTHOOK: query: explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME t2)) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME t1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key_int1)) (TOK_SELEXPR (TOK_TABLE_OR_COL key_string1)) (TOK_SELEXPR (TOK_FUNCTION upper (TOK_TABLE_OR_COL key_string2)) ref)))) Q1) (= (. (TOK_TABLE_OR_COL Q1) ref) (. (TOK_TABLE_OR_COL t2) key_string1))) (TOK_TABREF (TOK_TABNAME t3)) (= (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL Q1) ref) (. (TOK_TABLE_OR_COL t2) key_string1)) (. (TOK_TABLE_OR_COL t3) key_string1)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECTDI (TOK_SELEXPR (. (TOK_TABLE_OR_COL Q1) ref)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL t2) key_int1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL t3) key_string1))))) + +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-1 depends on stages: Stage-3 + Stage-2 depends on stages: Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-3 + Map Reduce + Alias -> Map Operator Tree: + q1:t1 + TableScan + alias: t1 + Select Operator + expressions: upper(key_string2)=upper(t1.key_string2) + outputColumnNames: _col2 + Reduce Output Operator + key expressions: _col2=upper(t1.key_string2) + sort order: + + Map-reduce partition columns: _col2=upper(t1.key_string2) + tag: 1 + value expressions: _col2=upper(t1.key_string2) + t2 + TableScan + alias: t2 + Reduce Output Operator + key expressions: key_string1=t2.key_string1 + sort order: + + Map-reduce partition columns: key_string1=t2.key_string1 + tag: 0 + value expressions: key_int1=t2.key_int1, key_string1=t2.key_string1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t2.key_int1, VALUE._col2=t2.key_string1] + 1 [VALUE._col2=upper(t1.key_string2)] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + outputColumnNames: _col0, _col2, _col8 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1) + sort order: + + Map-reduce partition columns: concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1) + tag: 0 + value expressions: _col0=t2.key_int1, _col8=upper(t1.key_string2) + t3 + TableScan + alias: t3 + Reduce Output Operator + key expressions: key_string1=t3.key_string1 + sort order: + + Map-reduce partition columns: key_string1=t3.key_string1 + tag: 1 + value expressions: key_string1=t3.key_string1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t2.key_int1, VALUE._col8=upper(t1.key_string2)] + 1 [VALUE._col2=t3.key_string1] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + outputColumnNames: _col0, _col8, _col11 + Select Operator + expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + outputColumnNames: _col8, _col0, _col11 + Group By Operator + bucketGroup: false + keys: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + 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: + file:/tmp/navis/hive_2011-08-05_00-39-42_303_4081806202832708501/-mr-10002 + Reduce Output Operator + key expressions: _col0=upper(t1.key_string2), _col1=t2.key_int1, _col2=t3.key_string1 + sort order: +++ + Map-reduce partition columns: _col0=upper(t1.key_string2), _col1=t2.key_int1, _col2=t3.key_string1 + tag: -1 + Reduce Operator Tree: + Group By Operator + bucketGroup: false + keys: KEY._col0=upper(t1.key_string2), KEY._col1=t2.key_int1, KEY._col2=t3.key_string1 + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0=upper(t1.key_string2), _col1=t2.key_int1, _col2=t3.key_string1 + 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 distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1 +PREHOOK: type: QUERY +POSTHOOK: query: explain + select distinct Q1.ref, t2.key_int1, t3.key_string1 from t2 + join (select key_int1, key_string1, upper(key_string2) as ref from t1) Q1 on Q1.ref=t2.key_string1 + join t3 on concat(Q1.ref, t2.key_string1)=t3.key_string1 +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME t2)) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME t1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key_int1)) (TOK_SELEXPR (TOK_TABLE_OR_COL key_string1)) (TOK_SELEXPR (TOK_FUNCTION upper (TOK_TABLE_OR_COL key_string2)) ref)))) Q1) (= (. (TOK_TABLE_OR_COL Q1) ref) (. (TOK_TABLE_OR_COL t2) key_string1))) (TOK_TABREF (TOK_TABNAME t3)) (= (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL Q1) ref) (. (TOK_TABLE_OR_COL t2) key_string1)) (. (TOK_TABLE_OR_COL t3) key_string1)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECTDI (TOK_SELEXPR (. (TOK_TABLE_OR_COL Q1) ref)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL t2) key_int1)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL t3) key_string1))))) + +STAGE DEPENDENCIES: + Stage-10 is a root stage , consists of Stage-13, Stage-14, Stage-3 + Stage-13 has a backup stage: Stage-3 + Stage-8 depends on stages: Stage-13 + Stage-7 depends on stages: Stage-3, Stage-8, Stage-9 , consists of Stage-11, Stage-12, Stage-1 + Stage-11 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-11 + Stage-2 depends on stages: Stage-1, Stage-5, Stage-6 + Stage-12 has a backup stage: Stage-1 + Stage-6 depends on stages: Stage-12 + Stage-1 + Stage-14 has a backup stage: Stage-3 + Stage-9 depends on stages: Stage-14 + Stage-3 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-10 + Conditional Operator + + Stage: Stage-13 + Map Reduce Local Work + Alias -> Map Local Tables: + q1:t1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + q1:t1 + TableScan + alias: t1 + Select Operator + expressions: upper(key_string2)=upper(t1.key_string2) + outputColumnNames: _col2 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + Position of Big Table: 0 + + Stage: Stage-8 + Map Reduce + Alias -> Map Operator Tree: + t2 + TableScan + alias: t2 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [key_int1=t2.key_int1, key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + outputColumnNames: _col0, _col2, _col8 + Position of Big Table: 0 + 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-7 + Conditional Operator + + Stage: Stage-11 + Map Reduce Local Work + Alias -> Map Local Tables: + t3 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + t3 + TableScan + alias: t3 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + Position of Big Table: 0 + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [_col0=t2.key_int1, _col8=upper(t1.key_string2)] + 1 [key_string1=t3.key_string1] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + outputColumnNames: _col0, _col8, _col11 + Position of Big Table: 0 + Select Operator + expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + outputColumnNames: _col8, _col0, _col11 + Group By Operator + bucketGroup: false + keys: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + file:/tmp/navis/hive_2011-08-05_00-39-42_601_7315592084061253060/-mr-10002 + Reduce Output Operator + key expressions: _col0=upper(t1.key_string2), _col1=t2.key_int1, _col2=t3.key_string1 + sort order: +++ + Map-reduce partition columns: _col0=upper(t1.key_string2), _col1=t2.key_int1, _col2=t3.key_string1 + tag: -1 + Reduce Operator Tree: + Group By Operator + bucketGroup: false + keys: KEY._col0=upper(t1.key_string2), KEY._col1=t2.key_int1, KEY._col2=t3.key_string1 + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0=upper(t1.key_string2), _col1=t2.key_int1, _col2=t3.key_string1 + 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-12 + Map Reduce Local Work + Alias -> Map Local Tables: + $INTNAME + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + $INTNAME + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + Position of Big Table: 1 + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: + t3 + TableScan + alias: t3 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [_col0=t2.key_int1, _col8=upper(t1.key_string2)] + 1 [key_string1=t3.key_string1] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + outputColumnNames: _col0, _col8, _col11 + Position of Big Table: 1 + Select Operator + expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + outputColumnNames: _col8, _col0, _col11 + Group By Operator + bucketGroup: false + keys: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + $INTNAME + Reduce Output Operator + key expressions: concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1) + sort order: + + Map-reduce partition columns: concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1) + tag: 0 + value expressions: _col0=t2.key_int1, _col8=upper(t1.key_string2) + t3 + TableScan + alias: t3 + Reduce Output Operator + key expressions: key_string1=t3.key_string1 + sort order: + + Map-reduce partition columns: key_string1=t3.key_string1 + tag: 1 + value expressions: key_string1=t3.key_string1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t2.key_int1, VALUE._col8=upper(t1.key_string2)] + 1 [VALUE._col2=t3.key_string1] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [concat(_col8, _col2)=concat(upper(t1.key_string2), t2.key_string1)] + 1 [key_string1=t3.key_string1] + outputColumnNames: _col0, _col8, _col11 + Select Operator + expressions: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + outputColumnNames: _col8, _col0, _col11 + Group By Operator + bucketGroup: false + keys: _col8=upper(t1.key_string2), _col0=t2.key_int1, _col11=t3.key_string1 + 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-14 + Map Reduce Local Work + Alias -> Map Local Tables: + t2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + t2 + TableScan + alias: t2 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + Position of Big Table: 1 + + Stage: Stage-9 + Map Reduce + Alias -> Map Operator Tree: + q1:t1 + TableScan + alias: t1 + Select Operator + expressions: upper(key_string2)=upper(t1.key_string2) + outputColumnNames: _col2 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [key_int1=t2.key_int1, key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + outputColumnNames: _col0, _col2, _col8 + Position of Big Table: 1 + 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: + q1:t1 + TableScan + alias: t1 + Select Operator + expressions: upper(key_string2)=upper(t1.key_string2) + outputColumnNames: _col2 + Reduce Output Operator + key expressions: _col2=upper(t1.key_string2) + sort order: + + Map-reduce partition columns: _col2=upper(t1.key_string2) + tag: 1 + value expressions: _col2=upper(t1.key_string2) + t2 + TableScan + alias: t2 + Reduce Output Operator + key expressions: key_string1=t2.key_string1 + sort order: + + Map-reduce partition columns: key_string1=t2.key_string1 + tag: 0 + value expressions: key_int1=t2.key_int1, key_string1=t2.key_string1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t2.key_int1, VALUE._col2=t2.key_string1] + 1 [VALUE._col2=upper(t1.key_string2)] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_string1=t2.key_string1] + 1 [_col2=upper(t1.key_string2)] + outputColumnNames: _col0, _col2, _col8 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1) +PREHOOK: type: QUERY +POSTHOOK: query: explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1)) (TOK_TABREF (TOK_TABNAME t2)) (= (. (TOK_TABLE_OR_COL t1) key_int1) (. (TOK_TABLE_OR_COL t2) key_int2)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)) (TOK_SELEXPR (TOK_FUNCTION sum (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1))) (TOK_SELEXPR (TOK_FUNCTIONDI sum (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)))) (TOK_GROUPBY (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)))) + +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: + t1 + TableScan + alias: t1 + Reduce Output Operator + key expressions: key_int1=t1.key_int1 + sort order: + + Map-reduce partition columns: key_int1=t1.key_int1 + tag: 0 + value expressions: key_int1=t1.key_int1 + t2 + TableScan + alias: t2 + Reduce Output Operator + key expressions: key_int2=t2.key_int2 + sort order: + + Map-reduce partition columns: key_int2=t2.key_int2 + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t1.key_int1] + 1 [] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + outputColumnNames: _col0 + Select Operator + expressions: _col0=t1.key_int1 + 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: + file:/tmp/navis/hive_2011-08-05_00-39-44_752_6450120344595000079/-mr-10002 + Reduce Output Operator + key expressions: (_col0 + 1)=(t1.key_int1 + 1) + sort order: + + Map-reduce partition columns: (_col0 + 1)=(t1.key_int1 + 1) + tag: -1 + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(KEY._col1:0._col0)=sum((t1.key_int1 + 1)) + expr: sum(DISTINCT KEY._col1:0._col0)=sum(DISTINCT (t1.key_int1 + 1)) + bucketGroup: false + keys: KEY._col1:0._col0=(t1.key_int1 + 1) + mode: complete + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0=(t1.key_int1 + 1), _col1=sum((t1.key_int1 + 1)), _col2=sum(DISTINCT (t1.key_int1 + 1)) + 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 (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1) +PREHOOK: type: QUERY +POSTHOOK: query: explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1)) (TOK_TABREF (TOK_TABNAME t2)) (= (. (TOK_TABLE_OR_COL t1) key_int1) (. (TOK_TABLE_OR_COL t2) key_int2)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)) (TOK_SELEXPR (TOK_FUNCTION sum (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1))) (TOK_SELEXPR (TOK_FUNCTIONDI sum (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)))) (TOK_GROUPBY (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)))) + +STAGE DEPENDENCIES: + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-2 depends on stages: Stage-1, Stage-4, Stage-5 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + t2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + t2 + TableScan + alias: t2 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + Position of Big Table: 0 + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: + t1 + TableScan + alias: t1 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [key_int1=t1.key_int1] + 1 [] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: _col0=t1.key_int1 + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + file:/tmp/navis/hive_2011-08-05_00-39-44_940_2359981358288946241/-mr-10002 + Reduce Output Operator + key expressions: (_col0 + 1)=(t1.key_int1 + 1) + sort order: + + Map-reduce partition columns: (_col0 + 1)=(t1.key_int1 + 1) + tag: -1 + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(KEY._col1:0._col0)=sum((t1.key_int1 + 1)) + expr: sum(DISTINCT KEY._col1:0._col0)=sum(DISTINCT (t1.key_int1 + 1)) + bucketGroup: false + keys: KEY._col1:0._col0=(t1.key_int1 + 1) + mode: complete + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0=(t1.key_int1 + 1), _col1=sum((t1.key_int1 + 1)), _col2=sum(DISTINCT (t1.key_int1 + 1)) + 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-8 + Map Reduce Local Work + Alias -> Map Local Tables: + t1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + t1 + TableScan + alias: t1 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + Position of Big Table: 1 + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + t2 + TableScan + alias: t2 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [key_int1=t1.key_int1] + 1 [] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + outputColumnNames: _col0 + Position of Big Table: 1 + Select Operator + expressions: _col0=t1.key_int1 + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + t1 + TableScan + alias: t1 + Reduce Output Operator + key expressions: key_int1=t1.key_int1 + sort order: + + Map-reduce partition columns: key_int1=t1.key_int1 + tag: 0 + value expressions: key_int1=t1.key_int1 + t2 + TableScan + alias: t2 + Reduce Output Operator + key expressions: key_int2=t2.key_int2 + sort order: + + Map-reduce partition columns: key_int2=t2.key_int2 + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t1.key_int1] + 1 [] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + outputColumnNames: _col0 + Select Operator + expressions: _col0=t1.key_int1 + outputColumnNames: _col0 + File Output Operator + compressed: false + GlobalTableId: 0 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + + Stage: Stage-0 + Fetch Operator + limit: -1 + + +PREHOOK: query: explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1) +PREHOOK: type: QUERY +POSTHOOK: query: explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1)) (TOK_TABREF (TOK_TABNAME t2)) (= (. (TOK_TABLE_OR_COL t1) key_int1) (. (TOK_TABLE_OR_COL t2) key_int2)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)) (TOK_SELEXPR (TOK_FUNCTION sum (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1))) (TOK_SELEXPR (TOK_FUNCTIONDI sum (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)))) (TOK_GROUPBY (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)))) + +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: + t1 + TableScan + alias: t1 + Reduce Output Operator + key expressions: key_int1=t1.key_int1 + sort order: + + Map-reduce partition columns: key_int1=t1.key_int1 + tag: 0 + value expressions: key_int1=t1.key_int1 + t2 + TableScan + alias: t2 + Reduce Output Operator + key expressions: key_int2=t2.key_int2 + sort order: + + Map-reduce partition columns: key_int2=t2.key_int2 + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t1.key_int1] + 1 [] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + outputColumnNames: _col0 + Select Operator + expressions: _col0=t1.key_int1 + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: sum((_col0 + 1))=sum((t1.key_int1 + 1)) + expr: sum(DISTINCT (_col0 + 1))=sum(DISTINCT (t1.key_int1 + 1)) + bucketGroup: false + keys: (_col0 + 1)=(t1.key_int1 + 1) + 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: + file:/tmp/navis/hive_2011-08-05_00-39-45_524_8765351967380684577/-mr-10002 + Reduce Output Operator + key expressions: _col0=(t1.key_int1 + 1) + sort order: + + Map-reduce partition columns: _col0=(t1.key_int1 + 1) + tag: -1 + value expressions: _col1=sum((t1.key_int1 + 1)), _col2=sum(DISTINCT (t1.key_int1 + 1)) + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0)=sum(sum((t1.key_int1 + 1))) + expr: sum(DISTINCT KEY._col1:0._col0)=sum(DISTINCT (t1.key_int1 + 1)) + bucketGroup: false + keys: KEY._col1:0._col0=(t1.key_int1 + 1) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0=(t1.key_int1 + 1), _col1=sum(sum((t1.key_int1 + 1))), _col2=sum(DISTINCT (t1.key_int1 + 1)) + 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 (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1) +PREHOOK: type: QUERY +POSTHOOK: query: explain + select (t1.key_int1+1), sum(t1.key_int1+1), sum(distinct t1.key_int1+1) from t1 + join t2 on t1.key_int1=t2.key_int2 + group by (t1.key_int1+1) +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME t1)) (TOK_TABREF (TOK_TABNAME t2)) (= (. (TOK_TABLE_OR_COL t1) key_int1) (. (TOK_TABLE_OR_COL t2) key_int2)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)) (TOK_SELEXPR (TOK_FUNCTION sum (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1))) (TOK_SELEXPR (TOK_FUNCTIONDI sum (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)))) (TOK_GROUPBY (+ (. (TOK_TABLE_OR_COL t1) key_int1) 1)))) + +STAGE DEPENDENCIES: + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-2 depends on stages: Stage-1, Stage-4, Stage-5 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + t2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + t2 + TableScan + alias: t2 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + Position of Big Table: 0 + + Stage: Stage-4 + Map Reduce + Alias -> Map Operator Tree: + t1 + TableScan + alias: t1 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [key_int1=t1.key_int1] + 1 [] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + outputColumnNames: _col0 + Position of Big Table: 0 + Select Operator + expressions: _col0=t1.key_int1 + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: sum((_col0 + 1))=sum((t1.key_int1 + 1)) + expr: sum(DISTINCT (_col0 + 1))=sum(DISTINCT (t1.key_int1 + 1)) + bucketGroup: false + keys: (_col0 + 1)=(t1.key_int1 + 1) + 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Map Reduce + Alias -> Map Operator Tree: + file:/tmp/navis/hive_2011-08-05_00-39-45_680_9068384874031751469/-mr-10002 + Reduce Output Operator + key expressions: _col0=(t1.key_int1 + 1) + sort order: + + Map-reduce partition columns: _col0=(t1.key_int1 + 1) + tag: -1 + value expressions: _col1=sum((t1.key_int1 + 1)), _col2=sum(DISTINCT (t1.key_int1 + 1)) + Reduce Operator Tree: + Group By Operator + aggregations: + expr: sum(VALUE._col0)=sum(sum((t1.key_int1 + 1))) + expr: sum(DISTINCT KEY._col1:0._col0)=sum(DISTINCT (t1.key_int1 + 1)) + bucketGroup: false + keys: KEY._col1:0._col0=(t1.key_int1 + 1) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0=(t1.key_int1 + 1), _col1=sum(sum((t1.key_int1 + 1))), _col2=sum(DISTINCT (t1.key_int1 + 1)) + 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-8 + Map Reduce Local Work + Alias -> Map Local Tables: + t1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + t1 + TableScan + alias: t1 + HashTable Sink Operator + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + Position of Big Table: 1 + + Stage: Stage-5 + Map Reduce + Alias -> Map Operator Tree: + t2 + TableScan + alias: t2 + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [key_int1=t1.key_int1] + 1 [] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + outputColumnNames: _col0 + Position of Big Table: 1 + Select Operator + expressions: _col0=t1.key_int1 + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: sum((_col0 + 1))=sum((t1.key_int1 + 1)) + expr: sum(DISTINCT (_col0 + 1))=sum(DISTINCT (t1.key_int1 + 1)) + bucketGroup: false + keys: (_col0 + 1)=(t1.key_int1 + 1) + 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Map Reduce + Alias -> Map Operator Tree: + t1 + TableScan + alias: t1 + Reduce Output Operator + key expressions: key_int1=t1.key_int1 + sort order: + + Map-reduce partition columns: key_int1=t1.key_int1 + tag: 0 + value expressions: key_int1=t1.key_int1 + t2 + TableScan + alias: t2 + Reduce Output Operator + key expressions: key_int2=t2.key_int2 + sort order: + + Map-reduce partition columns: key_int2=t2.key_int2 + tag: 1 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 [VALUE._col0=t1.key_int1] + 1 [] + filter predicates: + 0 [] + 1 [] + handleSkewJoin: false + keys: + 0 [key_int1=t1.key_int1] + 1 [key_int2=t2.key_int2] + outputColumnNames: _col0 + Select Operator + expressions: _col0=t1.key_int1 + outputColumnNames: _col0 + Group By Operator + aggregations: + expr: sum((_col0 + 1))=sum((t1.key_int1 + 1)) + expr: sum(DISTINCT (_col0 + 1))=sum(DISTINCT (t1.key_int1 + 1)) + bucketGroup: false + keys: (_col0 + 1)=(t1.key_int1 + 1) + 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-0 + Fetch Operator + limit: -1 + +