Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java =================================================================== --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (revision 1132812) +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (working copy) @@ -368,6 +368,7 @@ HIVEOPTBUCKETMAPJOIN("hive.optimize.bucketmapjoin", false), // optimize bucket map join HIVEOPTSORTMERGEBUCKETMAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false), // try to use sorted merge bucket map join HIVEOPTREDUCEDEDUPLICATION("hive.optimize.reducededuplication", true), + YSMARTOPTIMIZER("YSmartOptimizer", true), // Yin Turn on YSmart Optimizer for utilizing correlations // Indexes HIVEOPTINDEXFILTER_COMPACT_MINSIZE("hive.optimize.index.filter.compact.minsize", (long) 5 * 1024 * 1024 * 1024), // 5G Index: serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java =================================================================== --- serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java (revision 1132812) +++ serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java (working copy) @@ -147,6 +147,8 @@ */ @Override public Object deserialize(Writable field) throws SerDeException { + LazyBinaryStruct cachedLazyBinaryStruct = (LazyBinaryStruct) LazyBinaryFactory + .createLazyBinaryObject(cachedObjectInspector);//Yin if (byteArrayRef == null) { byteArrayRef = new ByteArrayRef(); } Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartOptimizerUtils.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartOptimizerUtils.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartOptimizerUtils.java (revision 0) @@ -0,0 +1,849 @@ +package org.apache.hadoop.hive.ql.optimizer; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.FilterOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.OperatorFactory; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.RowSchema; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.YSmartCompositeOperator; +import org.apache.hadoop.hive.ql.exec.YSmartFakeReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.YSmartReduceSinkOperator; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.optimizer.YSmartOptimizer_v2.YSmartCorrelation; +import org.apache.hadoop.hive.ql.parse.OpParseContext; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.RowResolver; +import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.TypeCheckProcFactory; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.FilterDesc; +import org.apache.hadoop.hive.ql.plan.ForwardDesc; +import org.apache.hadoop.hive.ql.plan.PlanUtils; +import org.apache.hadoop.hive.ql.plan.SelectDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.ql.plan.YSmartCompositeDesc; +import org.apache.hadoop.hive.ql.plan.YSmartDispatchDesc; +import org.apache.hadoop.hive.ql.plan.YSmartFakeReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.YSmartForwardDesc; +import org.apache.hadoop.hive.ql.plan.YSmartReduceSinkDesc; + + +public final class YSmartOptimizerUtils { + + public static boolean isExisted(ExprNodeDesc expr, ArrayList col_list){ + //System.out.println("typestring: " + expr.getTypeString() + ", exprString" + expr.getExprString()); + + for(ExprNodeDesc thisExpr: col_list){ + if(expr.getExprString().equals(thisExpr.getExprString())){ + return true; + } + } + return false; + } + + public static String getColumnName(Map opColumnExprMap, ExprNodeDesc expr) { + + for(Entry entry: opColumnExprMap.entrySet()){ + //System.out.println("in getColumnName:" + expr.getExprString() + "|" + entry.getValue().getExprString()); + + if(expr.getExprString().equals(entry.getValue().getExprString())){ + //System.out.println("in getColumnName: match"); + return entry.getKey(); + } + } + return null; + } + + + public static Operator unionUsedColumnsAndMakeNewSelect(ArrayList rsops, YSmartCorrelation correlation, + LinkedHashMap, Map> originalOpColumnExprMap, TableScanOperator input, ParseContext pGraphContext){ + //System.out.println((input == null)); + + ArrayList columnNames = new ArrayList(); + Map colExprMap = new HashMap(); + ArrayList col_list = new ArrayList(); + RowResolver out_rwsch = new RowResolver(); + boolean isSelectAll = false; + + int pos = 0; + for(ReduceSinkOperator rsop: rsops){ + + Operator curr = correlation.getBottom2TSops().get(rsop).get(0).getChildOperators().get(0); + while(true){ + if (curr.getName().equals("SEL")){ + SelectOperator selOp = (SelectOperator)curr; + //System.out.println("sel outputColNames: " + selOp.getConf().getOutputColumnNames() + " selOp.getColumnExprMap() != null" + (selOp.getColumnExprMap() != null)); + if(selOp.getColumnExprMap() != null){ + + for(Entry entry: selOp.getColumnExprMap().entrySet()){ + ExprNodeDesc expr = entry.getValue(); + if(!isExisted(expr, col_list)){ + col_list.add(expr); + String[] colRef = pGraphContext.getOpParseCtx().get(selOp).getRowResolver().getInvRslvMap().get(entry.getKey()); + String tabAlias = colRef[0]; + String colAlias = colRef[1]; + //String outputName = org.apache.hadoop.hive.ql.parse.SemanticAnalyzer.getColumnInternalName(pos); + String outputName = entry.getKey(); + out_rwsch.put(tabAlias, colAlias, new ColumnInfo( + outputName, expr.getTypeInfo(), tabAlias, false)); + pos++; + columnNames.add(outputName); + colExprMap.put(outputName, expr); + } + } + }else{ + //System.out.println("sel colList: " + selOp.getConf().getColList().toString()); + for(ExprNodeDesc expr: selOp.getConf().getColList()){ + if(!isExisted(expr, col_list)){ + col_list.add(expr); + String[] colRef = pGraphContext.getOpParseCtx().get(selOp).getRowResolver().getInvRslvMap().get(expr.getCols().get(0)); + //System.out.println("col:" + expr.getCols().get(0) + " colRef: " + colRef[0] + " " + colRef[1]); + String tabAlias = colRef[0]; + String colAlias = colRef[1]; + String outputName = expr.getCols().get(0); + out_rwsch.put(tabAlias, colAlias, new ColumnInfo( + outputName, expr.getTypeInfo(), tabAlias, false)); + columnNames.add(outputName); + colExprMap.put(outputName, expr); + pos++; + } + } + } + /* + + for(ExprNodeDesc expr: selOp.getConf().getColList()){ + if(!isExisted(expr, col_list)){ + col_list.add(expr); + + } + } + */ + break; + }else if (curr.getName().equals("FIL")){ + //FilterOperator filOp = (FilterOperator)curr; + //col_list.clear();// you need every column + isSelectAll = true; + break; + }else if(curr.getName().equals("RS")){ + ReduceSinkOperator thisRSop = (ReduceSinkOperator)curr; + for(ExprNodeDesc expr: thisRSop.getConf().getKeyCols()){ + if(!isExisted(expr, col_list)){ + col_list.add(expr); + assert expr.getCols().size() == 1; + String columnName = getColumnName(originalOpColumnExprMap.get(thisRSop), expr); + //System.out.println(columnName); + //System.out.println(inversedExpr2ColumnName.get(expr) + ", expr: " + expr.toString()); + //System.out.println( pGraphContext.getOpParseCtx().get(thisRSop).getRowResolver().getInvRslvMap().toString()); + String[] colRef = pGraphContext.getOpParseCtx().get(thisRSop).getRowResolver().getInvRslvMap().get(columnName); + //System.out.println("col:" + expr.getCols().get(0) + " colRef: " + colRef[0] + " " + colRef[1]); + String tabAlias = colRef[0]; + String colAlias = colRef[1]; + String outputName = expr.getCols().get(0); + out_rwsch.put(tabAlias, colAlias, new ColumnInfo( + outputName, expr.getTypeInfo(), tabAlias, false)); + columnNames.add(outputName); + colExprMap.put(outputName, expr); + pos++; + } + } + for(ExprNodeDesc expr: thisRSop.getConf().getValueCols()){ + if(!isExisted(expr, col_list)){ + col_list.add(expr); + assert expr.getCols().size() == 1; + String columnName = getColumnName(originalOpColumnExprMap.get(thisRSop), expr); + //System.out.println(columnName); + //System.out.println(inversedExpr2ColumnName.get(expr) + ", expr: " + expr.toString()); + //System.out.println( pGraphContext.getOpParseCtx().get(thisRSop).getRowResolver().getInvRslvMap().toString()); + String[] colRef = pGraphContext.getOpParseCtx().get(thisRSop).getRowResolver().getInvRslvMap().get(columnName); + //System.out.println("col:" + expr.getCols().get(0) + " colRef: " + colRef[0] + " " + colRef[1]); + String tabAlias = colRef[0]; + String colAlias = colRef[1]; + String outputName = expr.getCols().get(0); + out_rwsch.put(tabAlias, colAlias, new ColumnInfo( + outputName, expr.getTypeInfo(), tabAlias, false)); + columnNames.add(outputName); + colExprMap.put(outputName, expr); + pos++; + } + } + + break; + }else{ + curr = curr.getChildOperators().get(0); + } + + } + + + } + + Operator output; + if (isSelectAll){ + output = input; + }else{ + System.out.println(col_list.toString()); + System.out.println(columnNames.toString()); + + if(colExprMap != null) { + System.out.println(colExprMap.toString()); + }else{ + System.out.println("null"); + } + output = putOpInsertMap(OperatorFactory.getAndMakeChild( + new SelectDesc(col_list, columnNames, false), new RowSchema( + out_rwsch.getColumnInfos()), input), out_rwsch, pGraphContext.getOpParseCtx()); + output.setColumnExprMap(colExprMap); + output.setChildOperators(Utilities.makeList()); + + } + + + return output; + } + + + public static Operator putOpInsertMap(Operator op, + RowResolver rr, LinkedHashMap, OpParseContext> opParseCtx) { + OpParseContext ctx = new OpParseContext(rr); + opParseCtx.put(op, ctx); + op.augmentPlan(); + return op; + } + + public static HashMap, String> getAliasIDtTopOps(HashMap> topOps){ + HashMap, String> aliasIDtTopOps = new HashMap, String>(); + for(Entry> entry: topOps.entrySet()){ + assert !aliasIDtTopOps.containsKey(entry.getValue()); + aliasIDtTopOps.put(entry.getValue(), entry.getKey()); + + } + return aliasIDtTopOps; + } + + public static ArrayList findPeerReduceSinkOperators(ReduceSinkOperator op){ + + ArrayList peerReduceSinkOperators = new ArrayList(); + + List> children = op.getChildOperators(); + assert children.size() == 1; + + for(Operator parent: children.get(0).getParentOperators()){ + assert (parent instanceof ReduceSinkOperator); + peerReduceSinkOperators.add((ReduceSinkOperator)parent); + } + + return peerReduceSinkOperators; + } + + public static ArrayList findPeerFakeReduceSinkOperators(YSmartFakeReduceSinkOperator op){ + + ArrayList peerReduceSinkOperators = new ArrayList(); + + List> children = op.getChildOperators(); + assert children.size() == 1; + + for(Operator parent: children.get(0).getParentOperators()){ + assert (parent instanceof ReduceSinkOperator); + peerReduceSinkOperators.add((YSmartFakeReduceSinkOperator)parent); + } + + return peerReduceSinkOperators; + } + + // find how many layer's of Fake reduce sink + public static int getPostComputationDepth(YSmartCorrelation correlation){ + int depth = 0; + for(ReduceSinkOperator rsop: correlation.getBottomReduceSinkOperators()){ + ReduceSinkOperator op = rsop; + int layer = 0; + while(!correlation.getTopReduceSinkOperators().contains(op)){ + assert correlation.getDown2upRSops().get(op).size() == 1; + op = correlation.getDown2upRSops().get(op).get(0); + layer++; + } + if(layer > depth){ + depth = layer; + } + } + assert depth >= 1; + return depth; + } + + public static ParseContext applyCorrelation(YSmartCorrelation correlation, ParseContext inputpGraphContext, + LinkedHashMap, Map> originalOpColumnExprMap, + LinkedHashMap, RowResolver> originalOpRowResolver){ + + ParseContext pGraphContext = inputpGraphContext; + + Operator curr; + + + // 1: Create table scan operator + HashMap oldTSOP2newTSOP = new HashMap(); + + HashMap> oldTopOps = pGraphContext.getTopOps(); + HashMap, String> oldAliasIDtTopOps = getAliasIDtTopOps(oldTopOps); + HashMap oldTopToTable = pGraphContext.getTopToTable(); + + HashMap> addedTopOps = new HashMap>(); + HashMap addedTopToTable = new HashMap(); + + for(Entry> entry: correlation.getTable2CorrelatedTSops().entrySet()){ + TableScanOperator oldTSop = entry.getValue().get(0); + TableScanDesc tsDesc = new TableScanDesc(oldTSop.getConf().getAlias(), oldTSop.getConf().getVirtualCols()); + OpParseContext opParseCtx= pGraphContext.getOpParseCtx().get(oldTSop); + Operator top = putOpInsertMap(OperatorFactory.get(tsDesc, + new RowSchema(opParseCtx.getRowResolver().getColumnInfos())), + opParseCtx.getRowResolver(), pGraphContext.getOpParseCtx()); + top.setParentOperators(null); + top.setChildOperators(Utilities.makeList()); + + + for(TableScanOperator tsop: entry.getValue()){ + addedTopOps.put(oldAliasIDtTopOps.get(tsop), top); + addedTopToTable.put((TableScanOperator) top, oldTopToTable.get(tsop)); + oldTSOP2newTSOP.put(tsop, (TableScanOperator)top); + } + + } + + //System.out.println(addedTopOps.toString()); + //System.out.println(addedTopToTable.toString()); + //System.out.println(oldTSOP2newTSOP.toString()); + int postComputationDepth = getPostComputationDepth(correlation); + System.out.println("postComputationDepth: " + postComputationDepth); + ArrayList> childrenOfDispatch = new ArrayList>(); + for(ReduceSinkOperator rsop: correlation.getBottomReduceSinkOperators()){ + if(correlation.getTopReduceSinkOperators().contains(rsop)){ + Operator child = rsop.getChildOperators().get(0); + //YSmartForwardDesc forwardCtx = new YSmartForwardDesc(child.getParentOperators().indexOf(rsop)); + //Operator forwardOp = OperatorFactory.get(ForwardDesc.class); + //forwardOp.setConf(forwardCtx); + + Operator startForwardOp = putOpInsertMap(OperatorFactory.get(new YSmartForwardDesc(child.getParentOperators().indexOf(rsop)), + new RowSchema(pGraphContext.getOpParseCtx().get(rsop).getRowResolver().getColumnInfos())), + pGraphContext.getOpParseCtx().get(rsop).getRowResolver(), pGraphContext.getOpParseCtx()); + + //Operator child = rsop.getChildOperators().get(0); + startForwardOp.setChildOperators(Utilities.makeList(child)); + startForwardOp.setParentOperators(Utilities.makeList(rsop)); + rsop.replaceChild(child, startForwardOp); + if (!childrenOfDispatch.contains(startForwardOp)){ + childrenOfDispatch.add(startForwardOp); + } + + Operator lastForwardOp = startForwardOp; + + for(int i=1; i forwardOp = putOpInsertMap(OperatorFactory.get(new YSmartForwardDesc(0), + new RowSchema(pGraphContext.getOpParseCtx().get(rsop).getRowResolver().getColumnInfos())), + pGraphContext.getOpParseCtx().get(rsop).getRowResolver(), pGraphContext.getOpParseCtx()); + + //Operator child = rsop.getChildOperators().get(0); + lastForwardOp.setChildOperators(Utilities.makeList(forwardOp)); + forwardOp.setParentOperators(Utilities.makeList(lastForwardOp)); + lastForwardOp = forwardOp; + } + + child.replaceParent(rsop, lastForwardOp); + + }else{ + Operator op = rsop.getChildOperators().get(0); + if (!childrenOfDispatch.contains(op)){ + childrenOfDispatch.add(op); + } + } + + + } + + int opTag = 0; + HashMap opPlan2YsmartReduceSinkOps = new HashMap(); + for(Entry> entry: correlation.getTable2CorrelatedRSops().entrySet()){ + //System.out.println(correlation.getBottom2TSops().get(entry.getValue().get(0)).get(0)); + //System.out.println(oldTSOP2newTSOP.get(correlation.getBottom2TSops().get(entry.getValue().get(0)).get(0))); + + // 2: Create select operator for shared op plans + curr = unionUsedColumnsAndMakeNewSelect(entry.getValue(), correlation, originalOpColumnExprMap, + oldTSOP2newTSOP.get(correlation.getBottom2TSops().get(entry.getValue().get(0)).get(0)), pGraphContext); + // 3: Create YSmartCompositeOperator, YSmartReduceSinkOperator + curr = createYSmartCompositeReducesinkOperaotr( + correlation.getTable2CorrelatedTSops().get(entry.getKey()), entry.getValue(), correlation, curr, pGraphContext, + childrenOfDispatch, entry.getKey(), originalOpColumnExprMap, opTag, originalOpRowResolver); + + opPlan2YsmartReduceSinkOps.put(new Integer(opTag), (YSmartReduceSinkOperator)curr); + printOpInfo(curr); + opTag++; + } + + + // 4: Create YSmartDispatchOperator for shared op plans + RowResolver outputRS = new RowResolver(); + List> ysmartReduceSinkOps = new ArrayList>(); + for(Entry entry: opPlan2YsmartReduceSinkOps.entrySet()){ + Integer opTagInteger = entry.getKey(); + curr = entry.getValue(); + ysmartReduceSinkOps.add((YSmartReduceSinkOperator)curr); + + RowResolver inputRS = pGraphContext.getOpParseCtx().get(curr).getRowResolver(); + for(Entry> e1: inputRS.getRslvMap().entrySet()){ + for(Entry e2: e1.getValue().entrySet()){ + outputRS.put(e1.getKey(), e2.getKey(), e2.getValue()); + } + + } + + } + + Operator dispatchOp = putOpInsertMap(OperatorFactory.get( + new YSmartDispatchDesc(correlation.getDispatchConf(), correlation.getDispatchKeySelectDescConf(), correlation.getDispatchValueSelectDescConf()), + new RowSchema(outputRS.getColumnInfos())), + outputRS, pGraphContext.getOpParseCtx()); + + dispatchOp.setParentOperators(ysmartReduceSinkOps); + for(Operator thisOp: ysmartReduceSinkOps){ + thisOp.setChildOperators(Utilities.makeList(dispatchOp)); + } + + // 5: Replace the old plan in the original plan tree with new plan + + HashSet> processed = new HashSet>(); + //ArrayList> dispatchOpChildren = new ArrayList>(); + /* + for(Entry> entry: correlation.getTable2CorrelatedRSops().entrySet()){ + for(ReduceSinkOperator rsop: entry.getValue()){ + Operator op = rsop.getChildOperators().get(0); + if(processed.contains(op)){ + continue; + }else{ + op.setParentOperators(Utilities.makeList(dispatchOp)); + //dispatchOpChildren.add(op); + processed.add(op); + } + } + }*/ + for(Operator op: childrenOfDispatch){ + op.setParentOperators(Utilities.makeList(dispatchOp)); + } + dispatchOp.setChildOperators(childrenOfDispatch); + + HashMap> newTopOps = new HashMap>(); + for(Entry> entry: oldTopOps.entrySet()){ + if(addedTopOps.containsKey(entry.getKey())){ + newTopOps.put(entry.getKey(), addedTopOps.get(entry.getKey())); + }else{ + newTopOps.put(entry.getKey(), entry.getValue()); + } + } + pGraphContext.setTopOps(newTopOps); + + HashMap newTopToTable = new HashMap(); + for(Entry entry: oldTopToTable.entrySet()){ + if(addedTopToTable.containsKey(oldTSOP2newTSOP.get(entry.getKey()))){ + newTopToTable.put(oldTSOP2newTSOP.get(entry.getKey()), + addedTopToTable.get(oldTSOP2newTSOP.get(entry.getKey()))); + }else{ + newTopToTable.put(entry.getKey(), entry.getValue()); + } + } + pGraphContext.setTopToTable(newTopToTable); + + + // 6: Change JFC related RS to YSmartFakeReduceSinkOperator + + //System.out.println("^^^^^^^ getAllReduceSinkOperators ^^^^^^^"); + HashMap, ArrayList>> newParentsOfChildren = + new HashMap, ArrayList>>(); + for(ReduceSinkOperator rsop: correlation.getAllReduceSinkOperators()){ + System.out.println("name: " + rsop.getName() + ", id: " + rsop.getIdentifier()); + if(!correlation.getBottomReduceSinkOperators().contains(rsop)){ + Operator childOP = rsop.getChildOperators().get(0); + Operator parentOP = rsop.getParentOperators().get(0); + + Operator ysmartFakeRSOP = putOpInsertMap(OperatorFactory.get( + new YSmartFakeReduceSinkDesc(rsop.getConf()), + new RowSchema(pGraphContext.getOpParseCtx().get(rsop).getRowResolver().getColumnInfos())), + pGraphContext.getOpParseCtx().get(rsop).getRowResolver(), pGraphContext.getOpParseCtx()); + + ysmartFakeRSOP.setChildOperators(Utilities.makeList(childOP)); + ysmartFakeRSOP.setParentOperators(Utilities.makeList(parentOP)); + + System.out.println("parentOP " + parentOP.getIdentifier() + " parentOP #ofChildren:" + parentOP.getChildOperators().size()); + parentOP.getChildOperators().set(parentOP.getChildOperators().indexOf(rsop), ysmartFakeRSOP); + System.out.println("childOP " + childOP.getIdentifier() + " childOP #ofParents:" + childOP.getParentOperators().size() + + " childOP.getParentOperators().get(0):" + childOP.getParentOperators().get(0).getName()); + childOP.getParentOperators().set(childOP.getParentOperators().indexOf(rsop), ysmartFakeRSOP); + } + + + } + + + return pGraphContext; + /* + ArrayList> toWalk = new ArrayList>(); + toWalk.addAll(dispatchOp.getChildOperators()); + while(toWalk.size() != 0){ + int size = toWalk.size(); + ArrayList> toRemove = new ArrayList>(); + for(int i=0; i op: dispatchOp.getChildOperators()){ + + + Operator thisOp = op; + while(!correlation.getTopReduceSinkOperators().contains(thisOp)){ + + } + + } + */ + + } + + public static void printOpInfo(Operator op){ + + System.out.println("===PrintOpInfo Start==="); + System.out.println("op name: " + op.getName() + ", id: " + op.getIdentifier()); + if(op.getColumnExprMap() != null){ + System.out.println("op ColumnExprMap: " + op.getColumnExprMap().toString()); + } + if (op instanceof YSmartReduceSinkOperator){ + YSmartReduceSinkOperator ymrsop = (YSmartReduceSinkOperator)op; + System.out.println("ymrsop KeyCols: " + ymrsop.getConf().getKeyCols().toString()); + System.out.println("ymrsop OutputKeyColumnNames: " + ymrsop.getConf().getOutputKeyColumnNames().toString()); + System.out.println("ymrsop ValueCols: " + ymrsop.getConf().getValueCols().toString()); + System.out.println("ymrsop OutputValueColumnName: " + ymrsop.getConf().getOutputValueColumnNames().toString()); + } + System.out.println("===PrintOpInfo Done ==="); + + } + + + public static Operator createYSmartCompositeReducesinkOperaotr( + ArrayList tsops, ArrayList rsops, + YSmartCorrelation correlation, + Operator input, ParseContext pGraphContext, + ArrayList> childrenOfDispatch, String tableName, + LinkedHashMap, Map> originalOpColumnExprMap, int newTag, + LinkedHashMap, RowResolver> originalOpRowResolver){ + + // Create YSmartCompositeOperator + RowResolver inputRR = pGraphContext.getOpParseCtx().get(input).getRowResolver(); + ArrayList> tops = new ArrayList>(); + ArrayList> bottoms = new ArrayList>(); + ArrayList opTags = new ArrayList(); + + for(ReduceSinkOperator rsop: rsops){ + TableScanOperator tsop = correlation.getBottom2TSops().get(rsop).get(0); + Operator curr = tsop.getChildOperators().get(0); + if(curr == rsop){ + // no filter, just forward + ForwardDesc forwardCtx = new ForwardDesc(); + Operator forwardOp = OperatorFactory.get(ForwardDesc.class); + forwardOp.setConf(forwardCtx); + tops.add(forwardOp); + bottoms.add(forwardOp); + opTags.add(correlation.getBottomReduceSink2OpPlanMap().get(rsop)); + }else{ + // Add filter operator + FilterOperator currFilOp = null; + while(curr != rsop){ + if (curr.getName().equals("FIL")){ + FilterOperator fil = (FilterOperator)curr; + FilterDesc filterCtx = new FilterDesc(fil.getConf().getPredicate(), false); + Operator nowFilOp = OperatorFactory.get(FilterDesc.class); + nowFilOp.setConf(filterCtx); + if(currFilOp == null){ + currFilOp = (FilterOperator)nowFilOp; + tops.add(currFilOp); + }else{ + nowFilOp.setParentOperators(Utilities.makeList(currFilOp)); + currFilOp.setChildOperators(Utilities.makeList(nowFilOp)); + currFilOp = (FilterOperator) nowFilOp; + } + } + curr = curr.getChildOperators().get(0); + } + if (currFilOp == null){ + ForwardDesc forwardCtx = new ForwardDesc(); + Operator forwardOp = OperatorFactory.get(ForwardDesc.class); + forwardOp.setConf(forwardCtx); + tops.add(forwardOp); + bottoms.add(forwardOp); + } + else{ + bottoms.add(currFilOp); + } + opTags.add(correlation.getBottomReduceSink2OpPlanMap().get(rsop)); + + } + } + int[] opTagsArray = new int[opTags.size()]; + for(int i=0; i ycop = OperatorFactory.get(YSmartCompositeDesc.class); + YSmartCompositeDesc ycoCtx = new YSmartCompositeDesc(); + ycoCtx.setInternalNodes(tops, bottoms, opTagsArray); + Operator ycop = putOpInsertMap(OperatorFactory.getAndMakeChild(ycoCtx, + new RowSchema(inputRR.getColumnInfos()), input), + inputRR, pGraphContext.getOpParseCtx()); + //ycop.setConf(ycoCtx); + + //((YSmartCompositeOperator)ycop).getConf().setCorrespondingYRS(correspondingYRS); + + + + // Create YSmartReduceSinkOperator + ArrayList partitionCols = new ArrayList(); + ArrayList keyCols = new ArrayList(); + Map colExprMap = new HashMap(); + ArrayList keyOutputColumnNames = new ArrayList(); + ReduceSinkOperator firstRsop = rsops.get(0); + + RowResolver firstRsopRS = pGraphContext.getOpParseCtx().get(firstRsop).getRowResolver(); + RowResolver orginalFirstRsopRS = originalOpRowResolver.get(firstRsop); + RowResolver outputRS = new RowResolver(); + HashMap keyCol2ExprForDispatch = new HashMap(); + HashMap valueCol2ExprForDispatch = new HashMap(); + + for(ExprNodeDesc expr: firstRsop.getConf().getKeyCols()){ + assert expr instanceof ExprNodeColumnDesc; + ExprNodeColumnDesc encd = (ExprNodeColumnDesc)expr; + + + String ouputName = getColumnName(originalOpColumnExprMap.get(firstRsop), expr); + + ColumnInfo cinfo = orginalFirstRsopRS.getColumnInfos().get(orginalFirstRsopRS.getPosition(ouputName)); + + String col = SemanticAnalyzer.getColumnInternalName(keyCols.size()); + keyOutputColumnNames.add(col); + ColumnInfo newColInfo = new ColumnInfo(col, cinfo.getType(), tableName, cinfo + .getIsVirtualCol(), cinfo.isHiddenVirtualCol()); + + colExprMap.put(newColInfo.getInternalName(), expr); + + outputRS.put(tableName, newColInfo.getInternalName(), newColInfo); + keyCols.add(expr); + + keyCol2ExprForDispatch.put(encd.getColumn(), new ExprNodeColumnDesc(cinfo.getType(), col, tableName, + encd.getIsPartitionColOrVirtualCol())); + + + } + + + + //keyCols.addAll(firstRsop.getConf().getKeyCols()); + //rsop.getConf().getOutputKeyColumnNames() + + ArrayList valueCols = new ArrayList(); + ArrayList valueOutputColumnNames = new ArrayList(); + + + correlation.addOpPlanToDispatchConf(newTag); + correlation.addOpPlanToDispatchKeySelectDescConf(newTag); + correlation.addOpPlanToDispatchValueSelectDescConf(newTag); + + + for(ReduceSinkOperator rsop: rsops){ + + // gather info for common reduce sink and create selectDesc (for both key and value) for dispatcher + //Integer opPlan = correlation.getBottomReduceSink2OpPlanMap().get(rsop); + + + RowResolver rs = pGraphContext.getOpParseCtx().get(rsop).getRowResolver(); + RowResolver orginalRS = originalOpRowResolver.get(rsop); + Integer childOpIndex = childrenOfDispatch.indexOf(rsop.getChildOperators().get(0)); + + int outputTag = rsop.getConf().getTag(); + if(outputTag == -1){ + outputTag = 0; + } + + correlation.getDispatchConfForOpPlan(newTag).put(childOpIndex, outputTag); + + ArrayList thisKeyColsInDispatch = new ArrayList(); + ArrayList outputKeyNamesInDispatch = new ArrayList(); + for(ExprNodeDesc expr: rsop.getConf().getKeyCols()){ + assert expr instanceof ExprNodeColumnDesc; + ExprNodeColumnDesc encd = (ExprNodeColumnDesc)expr; + String outputName = getColumnName(originalOpColumnExprMap.get(rsop), expr); + ColumnInfo cinfo = orginalRS.getColumnInfos().get(orginalRS.getPosition(outputName)); + thisKeyColsInDispatch.add(keyCol2ExprForDispatch.get(encd.getColumn())); + String[] names = outputName.split("\\."); + outputKeyNamesInDispatch.add(names[1]); + //System.out.println("key:" + names[0] + " " + names[1]); + + } + correlation.getDispatchKeySelectDescConfForOpPlan(newTag).put(childOpIndex, new SelectDesc(thisKeyColsInDispatch, outputKeyNamesInDispatch, false)); + + ArrayList thisValueColsInDispatch = new ArrayList(); + ArrayList outputValueNamesInDispatch = new ArrayList(); + for(ExprNodeDesc expr: rsop.getConf().getValueCols()){ + + String outputName = getColumnName(originalOpColumnExprMap.get(rsop), expr); + System.out.println(rsop.getIdentifier() + " " + expr.getExprString() + " " + outputName); + System.out.println(rsop.getIdentifier() + " " + orginalRS.getColumnInfos().toString()); + + + ColumnInfo cinfo = orginalRS.getColumnInfos().get(orginalRS.getPosition(outputName)); + + + if(valueCol2ExprForDispatch.containsKey(expr.getExprString())){ + + }else{ + String col = SemanticAnalyzer.getColumnInternalName(keyCols.size() + valueCols.size()); + valueOutputColumnNames.add(col); + ColumnInfo newColInfo = new ColumnInfo(col, cinfo.getType(), tableName, cinfo + .getIsVirtualCol(), cinfo.isHiddenVirtualCol()); + colExprMap.put(newColInfo.getInternalName(), expr); + outputRS.put(tableName, newColInfo.getInternalName(), newColInfo); + valueCols.add(expr); + + valueCol2ExprForDispatch.put(expr.getExprString(), new ExprNodeColumnDesc(cinfo.getType(), col, tableName, + false)); + } + + thisValueColsInDispatch.add(valueCol2ExprForDispatch.get(expr.getExprString())); + String[] names = outputName.split("\\."); + outputValueNamesInDispatch.add(names[1]); + //System.out.println("value:" + names[0] + " " + names[1]); + //outputValueNamesInDispatch.add(ouputName); + } + + //System.out.println((correlation.getDispatchValueSelectDescConfForOpPlan(opPlan) == null) + " " + opPlan.intValue()); + correlation.getDispatchValueSelectDescConfForOpPlan(newTag). + put(childOpIndex, new SelectDesc(thisValueColsInDispatch, outputValueNamesInDispatch, false)); + + } + + YSmartReduceSinkOperator rsOp = null; + try { + rsOp = (YSmartReduceSinkOperator) putOpInsertMap( + OperatorFactory.getAndMakeChild(getReduceSinkDesc(keyCols, + keyCols.size(), valueCols, new ArrayList>(), + keyOutputColumnNames, valueOutputColumnNames, true, newTag, keyCols.size(), + -1), new RowSchema(outputRS + .getColumnInfos()), ycop), outputRS, pGraphContext.getOpParseCtx()); + rsOp.setColumnExprMap(colExprMap); + ((YSmartCompositeOperator)ycop).getConf().setCorrespondingYRS(rsOp); + } catch (SemanticException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + + + + return rsOp; + } + + + /** + * Create the ysmart reduce sink descriptor. + * + * @param keyCols + * The columns to be stored in the key + * @param numKeys number of distribution keys. Equals to group-by-key + * numbers usually. + * @param valueCols + * The columns to be stored in the value + * @param distinctColIndices + * column indices for distinct aggregates + * @param outputKeyColumnNames + * The output key columns names + * @param outputValueColumnNames + * The output value columns names + * @param tag + * The tag for this reducesink + * @param numPartitionFields + * The first numPartitionFields of keyCols will be partition columns. + * If numPartitionFields=-1, then partition randomly. + * @param numReducers + * The number of reducers, set to -1 for automatic inference based on + * input data size. + * @return The YSmartReduceSinkDesc object. + */ + public static YSmartReduceSinkDesc getReduceSinkDesc( + ArrayList keyCols, int numKeys, + ArrayList valueCols, + List> distinctColIndices, + List outputKeyColumnNames, List outputValueColumnNames, + boolean includeKey, int tag, + int numPartitionFields, int numReducers) throws SemanticException { + ArrayList partitionCols = null; + + if (numPartitionFields >= keyCols.size()) { + partitionCols = keyCols; + } else if (numPartitionFields >= 0) { + partitionCols = new ArrayList(numPartitionFields); + for (int i = 0; i < numPartitionFields; i++) { + partitionCols.add(keyCols.get(i)); + } + } else { + // numPartitionFields = -1 means random partitioning + partitionCols = new ArrayList(1); + partitionCols.add(TypeCheckProcFactory.DefaultExprProcessor + .getFuncExprNodeDesc("rand")); + } + + StringBuilder order = new StringBuilder(); + for (int i = 0; i < keyCols.size(); i++) { + order.append("+"); + } + + TableDesc keyTable = null; + TableDesc valueTable = null; + ArrayList outputKeyCols = new ArrayList(); + ArrayList outputValCols = new ArrayList(); + if (includeKey) { + keyTable = PlanUtils.getReduceKeyTableDesc(PlanUtils.getFieldSchemasFromColumnListWithLength( + keyCols, distinctColIndices, outputKeyColumnNames, numKeys, ""), + order.toString()); + outputKeyCols.addAll(outputKeyColumnNames); + } else { + keyTable = PlanUtils.getReduceKeyTableDesc(PlanUtils.getFieldSchemasFromColumnList( + keyCols, "reducesinkkey"),order.toString()); + for (int i = 0; i < keyCols.size(); i++) { + outputKeyCols.add("reducesinkkey" + i); + } + } + valueTable = PlanUtils.getReduceValueTableDesc(PlanUtils.getFieldSchemasFromColumnList( + valueCols, outputValueColumnNames, 0, "")); + outputValCols.addAll(outputValueColumnNames); + + return new YSmartReduceSinkDesc(keyCols, numKeys, valueCols, outputKeyCols, + distinctColIndices, outputValCols, + tag, partitionCols, numReducers, keyTable, + valueTable); + } + + +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMRRedSink1.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMRRedSink1.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMRRedSink1.java (revision 0) @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.optimizer; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; +import java.util.Stack; + +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.YSmartReduceSinkOperator; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.MapredWork; + +/** + * Processor for the rule - table scan followed by reduce sink. + */ +public class YSmartGenMRRedSink1 implements NodeProcessor { + + public YSmartGenMRRedSink1() { + } + + /** + * Reduce Scan encountered. + * + * @param nd + * the reduce sink operator encountered + * @param opProcCtx + * context + */ + public Object process(Node nd, Stack stack, NodeProcessorCtx opProcCtx, + Object... nodeOutputs) throws SemanticException { + YSmartReduceSinkOperator op = (YSmartReduceSinkOperator) nd; + GenMRProcContext ctx = (GenMRProcContext) opProcCtx; + + Map, GenMapRedCtx> mapCurrCtx = ctx + .getMapCurrCtx(); + GenMapRedCtx mapredCtx = mapCurrCtx.get(stack.get(stack.size() - 2)); + Task currTask = mapredCtx.getCurrTask(); + MapredWork currPlan = (MapredWork) currTask.getWork(); + Operator currTopOp = mapredCtx.getCurrTopOp(); + String currAliasId = mapredCtx.getCurrAliasId(); + Operator reducer = op.getChildOperators().get(0); + HashMap, Task> opTaskMap = ctx + .getOpTaskMap(); + Task opMapTask = opTaskMap.get(reducer); + + //System.out.println("currTopOp:" + currTopOp.toString() + "|" + currTopOp.getName()); //Yin + //System.out.println("currAliasId:" + currAliasId); + //System.out.println("currTask:" + currTask.toString()); + ctx.setCurrTopOp(currTopOp); + ctx.setCurrAliasId(currAliasId); + ctx.setCurrTask(currTask); + + // If the plan for this reducer does not exist, initialize the plan + if (opMapTask == null) { + if (currPlan.getReducer() == null) { + YSmartGenMapRedUtils.initPlan(op, ctx); + System.out.println("rule processing, currPlan reducer == null: children size" + op.getChildOperators().size());//Yin + } else { + YSmartGenMapRedUtils.splitPlan(op, ctx); + System.out.println("rule processing, currPlan reducer != null: children size" + op.getChildOperators().size());//Yin + } + } else { + // This will happen in case of joins. The current plan can be thrown away + // after being merged with the original plan + YSmartGenMapRedUtils.joinPlan(op, null, opMapTask, ctx, -1, false, false, false); + currTask = opMapTask; + ctx.setCurrTask(currTask); + System.out.println("rule processing, joinPlan: children size" + op.getChildOperators().size());//Yin + } + + mapCurrCtx.put(op, new GenMapRedCtx(ctx.getCurrTask(), ctx.getCurrTopOp(), + ctx.getCurrAliasId())); + return null; + } + +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java (revision 1132812) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java (working copy) @@ -44,7 +44,14 @@ */ public void initialize(HiveConf hiveConf) { transformations = new ArrayList(); + YSmartOptimizer_v2 ysmartOptimizer = new YSmartOptimizer_v2(); // Add the transformation that computes the lineage information. + if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.YSMARTOPTIMIZER)){ + transformations.add(ysmartOptimizer); // Add YSmart optimizer. + // The playground is used for trying different methods + //transformations.add(new YSmartOptimizer()); // Add YSmart optimizer + } + transformations.add(new Generator()); if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCP)) { transformations.add(new ColumnPruner()); @@ -72,6 +79,12 @@ if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATION)) { transformations.add(new ReduceSinkDeDuplication()); } + if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.YSMARTOPTIMIZER)){ + //transformations.add(new YSmartOptimizerPlayground()); // Add YSmart optimizer. + // The playground is used for trying different methods + transformations.add(ysmartOptimizer); // Add YSmart optimizer + } + } /** Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMapRedUtils.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMapRedUtils.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMapRedUtils.java (revision 0) @@ -0,0 +1,1007 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.optimizer; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.ConcurrentModificationException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; +import org.apache.hadoop.hive.ql.exec.JoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.OperatorFactory; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.TaskFactory; +import org.apache.hadoop.hive.ql.exec.UnionOperator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.YSmartReduceSinkOperator; +import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMRMapJoinCtx; +import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMRUnionCtx; +import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx; +import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner; +import org.apache.hadoop.hive.ql.optimizer.unionproc.UnionProcContext; +import org.apache.hadoop.hive.ql.optimizer.unionproc.UnionProcContext.UnionParseContext; +import org.apache.hadoop.hive.ql.parse.OpParseContext; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; +import org.apache.hadoop.hive.ql.parse.RowResolver; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.FetchWork; +import org.apache.hadoop.hive.ql.plan.FileSinkDesc; +import org.apache.hadoop.hive.ql.plan.MapJoinDesc; +import org.apache.hadoop.hive.ql.plan.MapredLocalWork; +import org.apache.hadoop.hive.ql.plan.MapredWork; +import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.ql.plan.PlanUtils; +import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.hive.ql.plan.YSmartReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.FilterDesc.sampleDesc; +import org.apache.hadoop.hive.ql.plan.MapredLocalWork.BucketMapJoinContext; + +/** + * General utility common functions for the Processor to convert operator into + * map-reduce tasks. + */ +public final class YSmartGenMapRedUtils { + private static Log LOG; + + static { + LOG = LogFactory.getLog("org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils"); + } + + /** + * Initialize the current plan by adding it to root tasks. + * + * @param op + * the reduce sink operator encountered + * @param opProcCtx + * processing context + */ + public static void initPlan(YSmartReduceSinkOperator op, GenMRProcContext opProcCtx) + throws SemanticException { + Operator reducer = op.getChildOperators().get(0); + Map, GenMapRedCtx> mapCurrCtx = opProcCtx.getMapCurrCtx(); + GenMapRedCtx mapredCtx = mapCurrCtx.get(op.getParentOperators().get(0)); + Task currTask = mapredCtx.getCurrTask(); + MapredWork plan = (MapredWork) currTask.getWork(); + HashMap, Task> opTaskMap = + opProcCtx.getOpTaskMap(); + Operator currTopOp = opProcCtx.getCurrTopOp(); + System.out.println("in initPlan: currTopOp, " + currTopOp.getIdentifier() + " " + currTopOp.getName()); + + opTaskMap.put(reducer, currTask); + plan.setReducer(reducer); + YSmartReduceSinkDesc desc = op.getConf(); + + plan.setNumReduceTasks(desc.getNumReducers()); + + List> rootTasks = opProcCtx.getRootTasks(); + + if (!rootTasks.contains(currTask)) { + rootTasks.add(currTask); + } + //if (reducer.getClass() == JoinOperator.class) { + plan.setNeedsTagging(true); + plan.setNeedsOpsTagging(true); + //} + + assert currTopOp != null; + List> seenOps = opProcCtx.getSeenOps(); + String currAliasId = opProcCtx.getCurrAliasId(); + + if (!seenOps.contains(currTopOp)) { + seenOps.add(currTopOp); + setTaskPlan(currAliasId, currTopOp, plan, false, opProcCtx); + } + + currTopOp = null; + currAliasId = null; + + opProcCtx.setCurrTask(currTask); + opProcCtx.setCurrTopOp(currTopOp); + opProcCtx.setCurrAliasId(currAliasId); + } + + public static void initMapJoinPlan( + Operator op, GenMRProcContext ctx, + boolean readInputMapJoin, boolean readInputUnion, boolean setReducer, int pos) throws SemanticException { + initMapJoinPlan(op, ctx, readInputMapJoin, readInputUnion, setReducer, pos, false); + } + + /** + * Initialize the current plan by adding it to root tasks. + * + * @param op + * the map join operator encountered + * @param opProcCtx + * processing context + * @param pos + * position of the parent + */ + public static void initMapJoinPlan(Operator op, + GenMRProcContext opProcCtx, boolean readInputMapJoin, + boolean readInputUnion, boolean setReducer, int pos, boolean createLocalPlan) + throws SemanticException { + Map, GenMapRedCtx> mapCurrCtx = opProcCtx.getMapCurrCtx(); + assert (((pos == -1) && (readInputMapJoin)) || (pos != -1)); + int parentPos = (pos == -1) ? 0 : pos; + GenMapRedCtx mapredCtx = mapCurrCtx.get(op.getParentOperators().get( + parentPos)); + Task currTask = mapredCtx.getCurrTask(); + MapredWork plan = (MapredWork) currTask.getWork(); + HashMap, Task> opTaskMap = + opProcCtx.getOpTaskMap(); + Operator currTopOp = opProcCtx.getCurrTopOp(); + + // The mapjoin has already been encountered. Some context must be stored + // about that + if (readInputMapJoin) { + AbstractMapJoinOperator currMapJoinOp = opProcCtx.getCurrMapJoinOp(); + assert currMapJoinOp != null; + boolean local = ((pos == -1) || (pos == (currMapJoinOp.getConf()).getPosBigTable())) ? + false : true; + + if (setReducer) { + Operator reducer = op.getChildOperators().get(0); + plan.setReducer(reducer); + opTaskMap.put(reducer, currTask); + if (reducer.getClass() == JoinOperator.class) { + plan.setNeedsTagging(true); + } + ReduceSinkDesc desc = (ReduceSinkDesc) op.getConf(); + plan.setNumReduceTasks(desc.getNumReducers()); + } else { + opTaskMap.put(op, currTask); + } + + if (!readInputUnion) { + GenMRMapJoinCtx mjCtx = opProcCtx.getMapJoinCtx(currMapJoinOp); + String taskTmpDir; + TableDesc tt_desc; + Operator rootOp; + + if (mjCtx.getOldMapJoin() == null || setReducer) { + taskTmpDir = mjCtx.getTaskTmpDir(); + tt_desc = mjCtx.getTTDesc(); + rootOp = mjCtx.getRootMapJoinOp(); + } else { + GenMRMapJoinCtx oldMjCtx = opProcCtx.getMapJoinCtx(mjCtx + .getOldMapJoin()); + taskTmpDir = oldMjCtx.getTaskTmpDir(); + tt_desc = oldMjCtx.getTTDesc(); + rootOp = oldMjCtx.getRootMapJoinOp(); + } + + setTaskPlan(taskTmpDir, taskTmpDir, rootOp, plan, local, tt_desc); + setupBucketMapJoinInfo(plan, currMapJoinOp, createLocalPlan); + } else { + initUnionPlan(opProcCtx, currTask, false); + } + + opProcCtx.setCurrMapJoinOp(null); + } else { + MapJoinDesc desc = (MapJoinDesc) op.getConf(); + + // The map is overloaded to keep track of mapjoins also + opTaskMap.put(op, currTask); + + List> rootTasks = opProcCtx.getRootTasks(); + rootTasks.add(currTask); + + assert currTopOp != null; + List> seenOps = opProcCtx.getSeenOps(); + String currAliasId = opProcCtx.getCurrAliasId(); + + seenOps.add(currTopOp); + boolean local = (pos == desc.getPosBigTable()) ? false : true; + setTaskPlan(currAliasId, currTopOp, plan, local, opProcCtx); + setupBucketMapJoinInfo(plan, (AbstractMapJoinOperator)op, createLocalPlan); + } + + opProcCtx.setCurrTask(currTask); + opProcCtx.setCurrTopOp(null); + opProcCtx.setCurrAliasId(null); + } + + private static void setupBucketMapJoinInfo(MapredWork plan, + AbstractMapJoinOperator currMapJoinOp, boolean createLocalPlan) { + if (currMapJoinOp != null) { + LinkedHashMap>> aliasBucketFileNameMapping = + currMapJoinOp.getConf().getAliasBucketFileNameMapping(); + if(aliasBucketFileNameMapping!= null) { + MapredLocalWork localPlan = plan.getMapLocalWork(); + if(localPlan == null) { + if(currMapJoinOp instanceof SMBMapJoinOperator) { + localPlan = ((SMBMapJoinOperator)currMapJoinOp).getConf().getLocalWork(); + } + if (localPlan == null && createLocalPlan) { + localPlan = new MapredLocalWork( + new LinkedHashMap>(), + new LinkedHashMap()); + } + } else { + //local plan is not null, we want to merge it into SMBMapJoinOperator's local work + if(currMapJoinOp instanceof SMBMapJoinOperator) { + MapredLocalWork smbLocalWork = ((SMBMapJoinOperator)currMapJoinOp).getConf().getLocalWork(); + if(smbLocalWork != null) { + localPlan.getAliasToFetchWork().putAll(smbLocalWork.getAliasToFetchWork()); + localPlan.getAliasToWork().putAll(smbLocalWork.getAliasToWork()); + } + } + } + + if(localPlan == null) { + return; + } + + if(currMapJoinOp instanceof SMBMapJoinOperator) { + plan.setMapLocalWork(null); + ((SMBMapJoinOperator)currMapJoinOp).getConf().setLocalWork(localPlan); + } else { + plan.setMapLocalWork(localPlan); + } + BucketMapJoinContext bucketMJCxt = new BucketMapJoinContext(); + localPlan.setBucketMapjoinContext(bucketMJCxt); + bucketMJCxt.setAliasBucketFileNameMapping(aliasBucketFileNameMapping); + bucketMJCxt.setBucketFileNameMapping(currMapJoinOp.getConf().getBucketFileNameMapping()); + localPlan.setInputFileChangeSensitive(true); + bucketMJCxt.setMapJoinBigTableAlias(currMapJoinOp.getConf().getBigTableAlias()); + bucketMJCxt.setBucketMatcherClass(org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class); + } + } + } + + /** + * Initialize the current union plan. + * + * @param op + * the reduce sink operator encountered + * @param opProcCtx + * processing context + */ + public static void initUnionPlan(ReduceSinkOperator op, + GenMRProcContext opProcCtx) throws SemanticException { + Operator reducer = op.getChildOperators().get(0); + Map, GenMapRedCtx> mapCurrCtx = opProcCtx.getMapCurrCtx(); + GenMapRedCtx mapredCtx = mapCurrCtx.get(op.getParentOperators().get(0)); + Task currTask = mapredCtx.getCurrTask(); + MapredWork plan = (MapredWork) currTask.getWork(); + HashMap, Task> opTaskMap = + opProcCtx.getOpTaskMap(); + + opTaskMap.put(reducer, currTask); + plan.setReducer(reducer); + ReduceSinkDesc desc = op.getConf(); + + plan.setNumReduceTasks(desc.getNumReducers()); + + if (reducer.getClass() == JoinOperator.class) { + plan.setNeedsTagging(true); + } + + initUnionPlan(opProcCtx, currTask, false); + } + + /* + * It is a idempotent function to add various intermediate files as the source + * for the union. The plan has already been created. + */ + public static void initUnionPlan(GenMRProcContext opProcCtx, + Task currTask, boolean local) { + MapredWork plan = (MapredWork) currTask.getWork(); + UnionOperator currUnionOp = opProcCtx.getCurrUnionOp(); + assert currUnionOp != null; + GenMRUnionCtx uCtx = opProcCtx.getUnionTask(currUnionOp); + assert uCtx != null; + + List taskTmpDirLst = uCtx.getTaskTmpDir(); + List tt_descLst = uCtx.getTTDesc(); + assert !taskTmpDirLst.isEmpty() && !tt_descLst.isEmpty(); + assert taskTmpDirLst.size() == tt_descLst.size(); + int size = taskTmpDirLst.size(); + assert local == false; + + for (int pos = 0; pos < size; pos++) { + String taskTmpDir = taskTmpDirLst.get(pos); + TableDesc tt_desc = tt_descLst.get(pos); + if (plan.getPathToAliases().get(taskTmpDir) == null) { + plan.getPathToAliases().put(taskTmpDir, new ArrayList()); + plan.getPathToAliases().get(taskTmpDir).add(taskTmpDir); + plan.getPathToPartitionInfo().put(taskTmpDir, + new PartitionDesc(tt_desc, null)); + plan.getAliasToWork().put(taskTmpDir, currUnionOp); + } + } + } + + + public static void joinPlan(Operator op, + Task oldTask, Task task, + GenMRProcContext opProcCtx, int pos, boolean split, + boolean readMapJoinData, boolean readUnionData) throws SemanticException { + joinPlan(op, oldTask, task, opProcCtx, pos, split, readMapJoinData, readUnionData, false); + } + + /** + * Merge the current task with the task for the current reducer. + * + * @param op + * operator being processed + * @param oldTask + * the old task for the current reducer + * @param task + * the current task for the current reducer + * @param opProcCtx + * processing context + * @param pos + * position of the parent in the stack + */ + public static void joinPlan(Operator op, + Task oldTask, Task task, + GenMRProcContext opProcCtx, int pos, boolean split, + boolean readMapJoinData, boolean readUnionData, boolean createLocalWork) + throws SemanticException { + Task currTask = task; + MapredWork plan = (MapredWork) currTask.getWork(); + Operator currTopOp = opProcCtx.getCurrTopOp(); + List> parTasks = null; + + // terminate the old task and make current task dependent on it + if (split) { + assert oldTask != null; + splitTasks(op, oldTask, currTask, opProcCtx, true, false, 0); + } else { + if ((oldTask != null) && (oldTask.getParentTasks() != null) + && !oldTask.getParentTasks().isEmpty()) { + parTasks = new ArrayList>(); + parTasks.addAll(oldTask.getParentTasks()); + + Object[] parTaskArr = parTasks.toArray(); + for (Object element : parTaskArr) { + ((Task) element).removeDependentTask(oldTask); + } + } + } + + + if (currTopOp != null) { + List> seenOps = opProcCtx.getSeenOps(); + String currAliasId = opProcCtx.getCurrAliasId(); + System.out.println("joinPlan, currTopOp:" + currTopOp.getName() + " "+ currTopOp.getIdentifier() + " " + (!seenOps.contains(currTopOp)));//Yin + + if (!seenOps.contains(currTopOp)) { + seenOps.add(currTopOp); + boolean local = false; + if (pos != -1) { + local = (pos == ((MapJoinDesc) op.getConf()).getPosBigTable()) ? false + : true; + } + setTaskPlan(currAliasId, currTopOp, plan, local, opProcCtx); + if(op instanceof AbstractMapJoinOperator) { + setupBucketMapJoinInfo(plan, (AbstractMapJoinOperator)op, createLocalWork); + } + } + currTopOp = null; + opProcCtx.setCurrTopOp(currTopOp); + } else if (opProcCtx.getCurrMapJoinOp() != null) { + AbstractMapJoinOperator mjOp = opProcCtx.getCurrMapJoinOp(); + if (readUnionData) { + initUnionPlan(opProcCtx, currTask, false); + } else { + GenMRMapJoinCtx mjCtx = opProcCtx.getMapJoinCtx(mjOp); + + // In case of map-join followed by map-join, the file needs to be + // obtained from the old map join + AbstractMapJoinOperator oldMapJoin = mjCtx.getOldMapJoin(); + String taskTmpDir = null; + TableDesc tt_desc = null; + Operator rootOp = null; + + boolean local = ((pos == -1) || (pos == (mjOp.getConf()) + .getPosBigTable())) ? false : true; + if (oldMapJoin == null) { + if (opProcCtx.getParseCtx().getListMapJoinOpsNoReducer().contains(mjOp) + || local || (oldTask != null) && (parTasks != null)) { + taskTmpDir = mjCtx.getTaskTmpDir(); + tt_desc = mjCtx.getTTDesc(); + rootOp = mjCtx.getRootMapJoinOp(); + } + } else { + GenMRMapJoinCtx oldMjCtx = opProcCtx.getMapJoinCtx(oldMapJoin); + assert oldMjCtx != null; + taskTmpDir = oldMjCtx.getTaskTmpDir(); + tt_desc = oldMjCtx.getTTDesc(); + rootOp = oldMjCtx.getRootMapJoinOp(); + } + + setTaskPlan(taskTmpDir, taskTmpDir, rootOp, plan, local, tt_desc); + setupBucketMapJoinInfo(plan, oldMapJoin, createLocalWork); + } + opProcCtx.setCurrMapJoinOp(null); + + if ((oldTask != null) && (parTasks != null)) { + for (Task parTask : parTasks) { + parTask.addDependentTask(currTask); + if(opProcCtx.getRootTasks().contains(currTask)) { + opProcCtx.getRootTasks().remove(currTask); + } + } + } + + } + + opProcCtx.setCurrTask(currTask); + } + + /** + * Split the current plan by creating a temporary destination. + * + * @param op + * the reduce sink operator encountered + * @param opProcCtx + * processing context + */ + public static void splitPlan(YSmartReduceSinkOperator op, GenMRProcContext opProcCtx) + throws SemanticException { + // Generate a new task + ParseContext parseCtx = opProcCtx.getParseCtx(); + MapredWork cplan = getMapRedWork(parseCtx.getConf()); + Task redTask = TaskFactory.get(cplan, parseCtx + .getConf()); + Operator reducer = op.getChildOperators().get(0); + + // Add the reducer + cplan.setReducer(reducer); + YSmartReduceSinkDesc desc = op.getConf(); + + cplan.setNumReduceTasks(new Integer(desc.getNumReducers())); + + HashMap, Task> opTaskMap = + opProcCtx.getOpTaskMap(); + opTaskMap.put(reducer, redTask); + Task currTask = opProcCtx.getCurrTask(); + + splitTasks(op, currTask, redTask, opProcCtx, true, false, 0); + opProcCtx.getRootOps().add(op); + } + + /** + * set the current task in the mapredWork. + * + * @param alias_id + * current alias + * @param topOp + * the top operator of the stack + * @param plan + * current plan + * @param local + * whether you need to add to map-reduce or local work + * @param opProcCtx + * processing context + */ + public static void setTaskPlan(String alias_id, + Operator topOp, MapredWork plan, boolean local, + GenMRProcContext opProcCtx) throws SemanticException { + setTaskPlan(alias_id, topOp, plan, local, opProcCtx, null); + } + + /** + * set the current task in the mapredWork. + * + * @param alias_id + * current alias + * @param topOp + * the top operator of the stack + * @param plan + * current plan + * @param local + * whether you need to add to map-reduce or local work + * @param opProcCtx + * processing context + * @param pList + * pruned partition list. If it is null it will be computed on-the-fly. + */ + public static void setTaskPlan(String alias_id, + Operator topOp, MapredWork plan, boolean local, + GenMRProcContext opProcCtx, PrunedPartitionList pList) throws SemanticException { + ParseContext parseCtx = opProcCtx.getParseCtx(); + Set inputs = opProcCtx.getInputs(); + + ArrayList partDir = new ArrayList(); + ArrayList partDesc = new ArrayList(); + + Path tblDir = null; + TableDesc tblDesc = null; + + PrunedPartitionList partsList = pList; + + if (partsList == null) { + try { + partsList = parseCtx.getOpToPartList().get((TableScanOperator)topOp); + if (partsList == null) { + //System.out.println("topOp:" + topOp.getIdentifier()); + partsList = PartitionPruner.prune(parseCtx.getTopToTable().get(topOp), + parseCtx.getOpToPartPruner().get(topOp), opProcCtx.getConf(), + alias_id, parseCtx.getPrunedPartitions()); + parseCtx.getOpToPartList().put((TableScanOperator)topOp, partsList); + } + } catch (SemanticException e) { + throw e; + } catch (HiveException e) { + LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e)); + throw new SemanticException(e.getMessage(), e); + } + } + + // Generate the map work for this alias_id + Set parts = null; + // pass both confirmed and unknown partitions through the map-reduce + // framework + + parts = partsList.getConfirmedPartns(); + parts.addAll(partsList.getUnknownPartns()); + PartitionDesc aliasPartnDesc = null; + try { + if (!parts.isEmpty()) { + aliasPartnDesc = Utilities.getPartitionDesc(parts.iterator().next()); + } + } catch (HiveException e) { + LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e)); + throw new SemanticException(e.getMessage(), e); + } + + // The table does not have any partitions + if (aliasPartnDesc == null) { + aliasPartnDesc = new PartitionDesc(Utilities.getTableDesc(parseCtx + .getTopToTable().get(topOp)), null); + + } + + plan.getAliasToPartnInfo().put(alias_id, aliasPartnDesc); + + for (Partition part : parts) { + if (part.getTable().isPartitioned()) { + inputs.add(new ReadEntity(part)); + } else { + inputs.add(new ReadEntity(part.getTable())); + } + + // Later the properties have to come from the partition as opposed + // to from the table in order to support versioning. + Path[] paths; + sampleDesc sampleDescr = parseCtx.getOpToSamplePruner().get(topOp); + + if (sampleDescr != null) { + paths = SamplePruner.prune(part, sampleDescr); + } else { + paths = part.getPath(); + } + + // is it a partitioned table ? + if (!part.getTable().isPartitioned()) { + assert ((tblDir == null) && (tblDesc == null)); + + tblDir = paths[0]; + tblDesc = Utilities.getTableDesc(part.getTable()); + } + + for (Path p : paths) { + if (p == null) { + continue; + } + String path = p.toString(); + if (LOG.isDebugEnabled()) { + LOG.debug("Adding " + path + " of table" + alias_id); + } + + partDir.add(p); + try { + partDesc.add(Utilities.getPartitionDesc(part)); + } catch (HiveException e) { + LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e)); + throw new SemanticException(e.getMessage(), e); + } + } + } + + Iterator iterPath = partDir.iterator(); + Iterator iterPartnDesc = partDesc.iterator(); + + if (!local) { + while (iterPath.hasNext()) { + assert iterPartnDesc.hasNext(); + String path = iterPath.next().toString(); + + PartitionDesc prtDesc = iterPartnDesc.next(); + + // Add the path to alias mapping + if (plan.getPathToAliases().get(path) == null) { + plan.getPathToAliases().put(path, new ArrayList()); + } + plan.getPathToAliases().get(path).add(alias_id); + plan.getPathToPartitionInfo().put(path, prtDesc); + if (LOG.isDebugEnabled()) { + LOG.debug("Information added for path " + path); + } + } + + assert plan.getAliasToWork().get(alias_id) == null; + plan.getAliasToWork().put(alias_id, topOp); + } else { + // populate local work if needed + MapredLocalWork localPlan = plan.getMapLocalWork(); + if (localPlan == null) { + localPlan = new MapredLocalWork( + new LinkedHashMap>(), + new LinkedHashMap()); + } + + assert localPlan.getAliasToWork().get(alias_id) == null; + assert localPlan.getAliasToFetchWork().get(alias_id) == null; + localPlan.getAliasToWork().put(alias_id, topOp); + if (tblDir == null) { + localPlan.getAliasToFetchWork().put( + alias_id, + new FetchWork(FetchWork.convertPathToStringArray(partDir), partDesc)); + } else { + localPlan.getAliasToFetchWork().put(alias_id, + new FetchWork(tblDir.toString(), tblDesc)); + } + plan.setMapLocalWork(localPlan); + } + } + + /** + * set the current task in the mapredWork. + * + * @param alias + * current alias + * @param topOp + * the top operator of the stack + * @param plan + * current plan + * @param local + * whether you need to add to map-reduce or local work + * @param tt_desc + * table descriptor + */ + public static void setTaskPlan(String path, String alias, + Operator topOp, MapredWork plan, boolean local, + TableDesc tt_desc) throws SemanticException { + + if(path == null || alias == null) { + return; + } + + if (!local) { + if (plan.getPathToAliases().get(path) == null) { + plan.getPathToAliases().put(path, new ArrayList()); + } + plan.getPathToAliases().get(path).add(alias); + plan.getPathToPartitionInfo().put(path, new PartitionDesc(tt_desc, null)); + plan.getAliasToWork().put(alias, topOp); + } else { + // populate local work if needed + MapredLocalWork localPlan = plan.getMapLocalWork(); + if (localPlan == null) { + localPlan = new MapredLocalWork( + new LinkedHashMap>(), + new LinkedHashMap()); + } + + assert localPlan.getAliasToWork().get(alias) == null; + assert localPlan.getAliasToFetchWork().get(alias) == null; + localPlan.getAliasToWork().put(alias, topOp); + localPlan.getAliasToFetchWork().put(alias, new FetchWork(alias, tt_desc)); + plan.setMapLocalWork(localPlan); + } + } + + /** + * set key and value descriptor. + * + * @param plan + * current plan + * @param topOp + * current top operator in the path + */ + public static void setKeyAndValueDesc(MapredWork plan, + Operator topOp) { + if (topOp == null) { + return; + } + + if (topOp instanceof ReduceSinkOperator) { + ReduceSinkOperator rs = (ReduceSinkOperator) topOp; + plan.setKeyDesc(rs.getConf().getKeySerializeInfo()); + int tag = Math.max(0, rs.getConf().getTag()); + List tagToSchema = plan.getTagToValueDesc(); + while (tag + 1 > tagToSchema.size()) { + tagToSchema.add(null); + } + tagToSchema.set(tag, rs.getConf().getValueSerializeInfo()); + } else { + List> children = topOp.getChildOperators(); + if (children != null) { + for (Operator op : children) { + setKeyAndValueDesc(plan, op); + } + } + } + } + + /** + * create a new plan and return. + * + * @return the new plan + */ + public static MapredWork getMapRedWork(HiveConf conf) { + MapredWork work = new MapredWork(); + // This code has been only added for testing + boolean mapperCannotSpanPartns = + conf.getBoolVar( + HiveConf.ConfVars.HIVE_MAPPER_CANNOT_SPAN_MULTIPLE_PARTITIONS); + work.setMapperCannotSpanPartns(mapperCannotSpanPartns); + work.setPathToAliases(new LinkedHashMap>()); + work.setPathToPartitionInfo(new LinkedHashMap()); + work.setAliasToWork(new LinkedHashMap>()); + work.setTagToValueDesc(new ArrayList()); + work.setReducer(null); + work.setHadoopSupportsSplittable( + conf.getBoolVar(HiveConf.ConfVars.HIVE_COMBINE_INPUT_FORMAT_SUPPORTS_SPLITTABLE)); + return work; + } + + /** + * insert in the map for the operator to row resolver. + * + * @param op + * operator created + * @param rr + * row resolver + * @param parseCtx + * parse context + */ + @SuppressWarnings("nls") + private static Operator putOpInsertMap( + Operator op, RowResolver rr, ParseContext parseCtx) { + OpParseContext ctx = new OpParseContext(rr); + parseCtx.getOpParseCtx().put(op, ctx); + return op; + } + + @SuppressWarnings("nls") + /** + * Merge the tasks - by creating a temporary file between them. + * @param op reduce sink operator being processed + * @param oldTask the parent task + * @param task the child task + * @param opProcCtx context + * @param setReducer does the reducer needs to be set + * @param pos position of the parent + **/ + public static void splitTasks(Operator op, + Task parentTask, + Task childTask, GenMRProcContext opProcCtx, + boolean setReducer, boolean local, int posn) throws SemanticException { + childTask.getWork(); + Operator currTopOp = opProcCtx.getCurrTopOp(); + + ParseContext parseCtx = opProcCtx.getParseCtx(); + parentTask.addDependentTask(childTask); + + // Root Task cannot depend on any other task, therefore childTask cannot be + // a root Task + List> rootTasks = opProcCtx.getRootTasks(); + if (rootTasks.contains(childTask)) { + rootTasks.remove(childTask); + } + + // generate the temporary file + Context baseCtx = parseCtx.getContext(); + String taskTmpDir = baseCtx.getMRTmpFileURI(); + + Operator parent = op.getParentOperators().get(posn); + TableDesc tt_desc = PlanUtils.getIntermediateFileTableDesc(PlanUtils + .getFieldSchemasFromRowSchema(parent.getSchema(), "temporarycol")); + + // Create a file sink operator for this file name + boolean compressIntermediate = parseCtx.getConf().getBoolVar( + HiveConf.ConfVars.COMPRESSINTERMEDIATE); + FileSinkDesc desc = new FileSinkDesc(taskTmpDir, tt_desc, + compressIntermediate); + if (compressIntermediate) { + desc.setCompressCodec(parseCtx.getConf().getVar( + HiveConf.ConfVars.COMPRESSINTERMEDIATECODEC)); + desc.setCompressType(parseCtx.getConf().getVar( + HiveConf.ConfVars.COMPRESSINTERMEDIATETYPE)); + } + Operator fs_op = putOpInsertMap(OperatorFactory + .get(desc, parent.getSchema()), null, parseCtx); + + // replace the reduce child with this operator + List> childOpList = parent + .getChildOperators(); + for (int pos = 0; pos < childOpList.size(); pos++) { + if (childOpList.get(pos) == op) { + childOpList.set(pos, fs_op); + break; + } + } + + List> parentOpList = + new ArrayList>(); + parentOpList.add(parent); + fs_op.setParentOperators(parentOpList); + + // 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 + .get(TableScanDesc.class, parent.getSchema()), rowResolver, parseCtx); + + childOpList = new ArrayList>(); + childOpList.add(op); + ts_op.setChildOperators(childOpList); + op.getParentOperators().set(posn, ts_op); + + Map, GenMapRedCtx> mapCurrCtx = opProcCtx.getMapCurrCtx(); + mapCurrCtx.put(ts_op, new GenMapRedCtx(childTask, null, null)); + + String streamDesc = taskTmpDir; + MapredWork cplan = (MapredWork) childTask.getWork(); + + if (setReducer) { + Operator reducer = op.getChildOperators().get(0); + + if (reducer.getClass() == JoinOperator.class) { + String origStreamDesc; + streamDesc = "$INTNAME"; + origStreamDesc = streamDesc; + int pos = 0; + while (cplan.getAliasToWork().get(streamDesc) != null) { + streamDesc = origStreamDesc.concat(String.valueOf(++pos)); + } + } + + // TODO: Allocate work to remove the temporary files and make that + // dependent on the redTask + if (reducer.getClass() == JoinOperator.class) { + cplan.setNeedsTagging(true); + } + } + + // Add the path to alias mapping + setTaskPlan(taskTmpDir, streamDesc, ts_op, cplan, local, tt_desc); + + // This can be cleaned up as a function table in future + if (op instanceof AbstractMapJoinOperator) { + AbstractMapJoinOperator mjOp = (AbstractMapJoinOperator) op; + opProcCtx.setCurrMapJoinOp(mjOp); + GenMRMapJoinCtx mjCtx = opProcCtx.getMapJoinCtx(mjOp); + if (mjCtx == null) { + mjCtx = new GenMRMapJoinCtx(taskTmpDir, tt_desc, ts_op, null); + } else { + mjCtx.setTaskTmpDir(taskTmpDir); + mjCtx.setTTDesc(tt_desc); + mjCtx.setRootMapJoinOp(ts_op); + } + opProcCtx.setMapJoinCtx(mjOp, mjCtx); + opProcCtx.getMapCurrCtx().put(parent, + new GenMapRedCtx(childTask, null, null)); + setupBucketMapJoinInfo(cplan, mjOp, false); + } + + currTopOp = null; + String currAliasId = null; + + opProcCtx.setCurrTopOp(currTopOp); + opProcCtx.setCurrAliasId(currAliasId); + opProcCtx.setCurrTask(childTask); + } + + public static void mergeMapJoinUnion(UnionOperator union, + GenMRProcContext ctx, int pos) throws SemanticException { + ParseContext parseCtx = ctx.getParseCtx(); + UnionProcContext uCtx = parseCtx.getUCtx(); + + UnionParseContext uPrsCtx = uCtx.getUnionParseContext(union); + assert uPrsCtx != null; + + Task currTask = ctx.getCurrTask(); + + GenMRUnionCtx uCtxTask = ctx.getUnionTask(union); + Task uTask = null; + + union.getParentOperators().get(pos); + MapredWork uPlan = null; + + // union is encountered for the first time + if (uCtxTask == null) { + uCtxTask = new GenMRUnionCtx(); + uPlan = YSmartGenMapRedUtils.getMapRedWork(parseCtx.getConf()); + uTask = TaskFactory.get(uPlan, parseCtx.getConf()); + uCtxTask.setUTask(uTask); + ctx.setUnionTask(union, uCtxTask); + } else { + uTask = uCtxTask.getUTask(); + uPlan = (MapredWork) uTask.getWork(); + } + + // If there is a mapjoin at position 'pos' + if (uPrsCtx.getMapJoinSubq(pos)) { + GenMRMapJoinCtx mjCtx = ctx.getMapJoinCtx(ctx.getCurrMapJoinOp()); + String taskTmpDir = mjCtx.getTaskTmpDir(); + if (uPlan.getPathToAliases().get(taskTmpDir) == null) { + uPlan.getPathToAliases().put(taskTmpDir, new ArrayList()); + uPlan.getPathToAliases().get(taskTmpDir).add(taskTmpDir); + uPlan.getPathToPartitionInfo().put(taskTmpDir, + new PartitionDesc(mjCtx.getTTDesc(), null)); + uPlan.getAliasToWork().put(taskTmpDir, mjCtx.getRootMapJoinOp()); + } + + for (Task t : currTask.getParentTasks()) { + t.addDependentTask(uTask); + } + try { + boolean notDone = true; + while (notDone) { + for (Task t : currTask.getParentTasks()) { + t.removeDependentTask(currTask); + } + notDone = false; + } + } catch (ConcurrentModificationException e) { + } + } else { + setTaskPlan(ctx.getCurrAliasId(), ctx.getCurrTopOp(), uPlan, false, ctx); + } + + ctx.setCurrTask(uTask); + ctx.setCurrAliasId(null); + ctx.setCurrTopOp(null); + ctx.setCurrMapJoinOp(null); + + ctx.getMapCurrCtx().put(union, + new GenMapRedCtx(ctx.getCurrTask(), null, null)); + } + + private YSmartGenMapRedUtils() { + // prevent instantiation + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMRRedSink2.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMRRedSink2.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartGenMRRedSink2.java (revision 0) @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.optimizer; + +import java.io.Serializable; +import java.util.Map; +import java.util.Stack; + +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx; +import org.apache.hadoop.hive.ql.parse.SemanticException; + +/** + * Processor for the rule - reduce sink followed by reduce sink. + */ +public class YSmartGenMRRedSink2 implements NodeProcessor { + + public YSmartGenMRRedSink2() { + } + + /** + * Reduce Scan encountered. + * + * @param nd + * the reduce sink operator encountered + * @param opProcCtx + * context + */ + public Object process(Node nd, Stack stack, NodeProcessorCtx opProcCtx, + Object... nodeOutputs) throws SemanticException { + ReduceSinkOperator op = (ReduceSinkOperator) nd; + GenMRProcContext ctx = (GenMRProcContext) opProcCtx; + + Map, GenMapRedCtx> mapCurrCtx = ctx + .getMapCurrCtx(); + GenMapRedCtx mapredCtx = mapCurrCtx.get(op.getParentOperators().get(0)); + Task currTask = mapredCtx.getCurrTask(); + Operator currTopOp = mapredCtx.getCurrTopOp(); + String currAliasId = mapredCtx.getCurrAliasId(); + Operator reducer = op.getChildOperators().get(0); + Map, Task> opTaskMap = ctx + .getOpTaskMap(); + Task opMapTask = opTaskMap.get(reducer); + + ctx.setCurrTopOp(currTopOp); + ctx.setCurrAliasId(currAliasId); + ctx.setCurrTask(currTask); + + + if (opMapTask == null) { + GenMapRedUtils.splitPlan(op, ctx); + } else { + GenMapRedUtils.joinPlan(op, currTask, opMapTask, ctx, -1, true, false, + false); + currTask = opMapTask; + ctx.setCurrTask(currTask); + } + + mapCurrCtx.put(op, new GenMapRedCtx(ctx.getCurrTask(), ctx.getCurrTopOp(), + ctx.getCurrAliasId())); + return null; + } + +} Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (revision 1132812) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (working copy) @@ -44,6 +44,7 @@ import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.YSmartReduceSinkOperator; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.Partition; @@ -823,7 +824,16 @@ tagToSchema.add(null); } tagToSchema.set(tag, rs.getConf().getValueSerializeInfo()); - } else { + } else if (topOp instanceof YSmartReduceSinkOperator) { //Yin + YSmartReduceSinkOperator rs = (YSmartReduceSinkOperator) topOp; + plan.setKeyDesc(rs.getConf().getKeySerializeInfo()); + int tag = Math.max(0, rs.getConf().getTag()); + List tagToSchema = plan.getTagToValueDesc(); + while (tag + 1 > tagToSchema.size()) { + tagToSchema.add(null); + } + tagToSchema.set(tag, rs.getConf().getValueSerializeInfo()); + }else { List> children = topOp.getChildOperators(); if (children != null) { for (Operator op : children) { Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartOptimizer_v2.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartOptimizer_v2.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/YSmartOptimizer_v2.java (revision 0) @@ -0,0 +1,1192 @@ +package org.apache.hadoop.hive.ql.optimizer; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Stack; +import java.util.Map.Entry; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.ColumnInfo; +import org.apache.hadoop.hive.ql.exec.GroupByOperator; +import org.apache.hadoop.hive.ql.exec.JoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; +import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; +import org.apache.hadoop.hive.ql.lib.Dispatcher; +import org.apache.hadoop.hive.ql.lib.GraphWalker; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.lib.Rule; +import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.OpParseContext; +import org.apache.hadoop.hive.ql.parse.ParseContext; +import org.apache.hadoop.hive.ql.parse.QB; +import org.apache.hadoop.hive.ql.parse.QBExpr; +import org.apache.hadoop.hive.ql.parse.QBJoinTree; +import org.apache.hadoop.hive.ql.parse.RowResolver; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.SelectDesc; +import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +/** + * Implementation of rule-based YSmart optimizer. This optimizer detect three kinds of + * correlations, Input Correlation (IC), Transit Correlation (TC) and Job Flow Correlation (JFC). + * After the optimization, the structure of a plan tree should be TS->YCO->SEL(select the union of columns + * which used by operations)->YRS->YSmartDispatcher->regular operations + */ + +public class YSmartOptimizer_v2 implements Transform { + + static final private Log LOG = LogFactory.getLog(YSmartOptimizer_v2.class.getName()); + static final private LogHelper console = new LogHelper(LOG); + private final HashMap AliastoTabName; + private final HashMap AliastoTab; + private final HashMap> ColumnMap; + + + + + + //private HashMap operatorToPartitionKey + public YSmartOptimizer_v2(){ + super(); + AliastoTabName = new HashMap(); + AliastoTab = new HashMap(); + ColumnMap = new HashMap>(); + pGraphContext = null; + } + + private void initializeAliastoTabNameMapping(QB qb){ + for (String alias: qb.getAliases()){ + AliastoTabName.put(alias, qb.getTabNameForAlias(alias)); + AliastoTab.put(alias, qb.getMetaData().getSrcForAlias(alias)); + } + for (String subqalias: qb.getSubqAliases()){ + QBExpr qbexpr = qb.getSubqForAlias(subqalias); + initializeAliastoTabNameMapping(qbexpr.getQB()); + } + } + + private void jobFlowCorrelationAnalyze(){ + + } + + // find next select operator before nearest reduce sink + private SelectOperator findNextSelect(Operator startPoint){ + Operator thisOp = startPoint; + while(true){ + if(thisOp.getName().compareTo("SEL") == 0) { + return (SelectOperator) thisOp; + } + else if (thisOp.getName().compareTo("RS") != 0){ + break; + } + else{ + if (thisOp.getChildOperators() != null) { + assert thisOp.getChildOperators().size() == 1; + thisOp = (Operator) thisOp.getChildOperators().get(0); + } + else{ + break; + } + } + } + return null; + } + + //find next select operator before nearest reduce sink + private ReduceSinkOperator findNextReduceSink(Operator startPoint){ + Operator thisOp = startPoint; + while(true){ + if(thisOp.getName().compareTo("RS") == 0) { + return (ReduceSinkOperator) thisOp; + } + else{ + if (thisOp.getChildOperators() != null) { + assert thisOp.getChildOperators().size() == 1; + thisOp = (Operator) thisOp.getChildOperators().get(0); + } + else{ + break; + } + } + } + return null; + } + + + @SuppressWarnings("unchecked") + public void transformTPCHQ17ShowCase(){ + HashMap> topOps = pGraphContext.getTopOps(); + HashMap topOpToTable = pGraphContext.getTopToTable(); + + System.out.println("topOps---------------------------------------------------------"); + for(Entry> entry: topOps.entrySet()){ + System.out.println(entry.getKey() + ": " + entry.getValue().getName() + " from " + topOpToTable.get((TableScanOperator)entry.getValue())); + } + + + + System.out.println("pGraphContext.getGroupOpToInputTables().entrySet()-------------"); + for(Entry> entry: pGraphContext.getGroupOpToInputTables().entrySet()){ + System.out.println(entry.getKey().getConf().getKeys().toString() + ": " + entry.getValue().toString()); + } + + + System.out.println("pGraphContext.getJoinContext().entrySet()----------------------"); + for(Entry entry: pGraphContext.getJoinContext().entrySet()){ + System.out.println(entry.getKey().getConf().getConds() + ": " + entry.getValue().toString()); + } + + //System.out.println("pGraphContext.getIdToTableNameMap().entrySet()-----------------"); + //for(Entry entry: pGraphContext.getIdToTableNameMap().entrySet()){ + // System.out.println(entry.getKey() + ": " + entry.getValue()); + //} + + + + System.out.println("tpchq17 show case done-----------------------------------------"); + } + + public static ExprNodeColumnDesc getStringColumn(String columnName) { + return new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, columnName, + "", false); + } + + protected ParseContext pGraphContext; + private LinkedHashMap, OpParseContext> opParseCtx; + private LinkedHashMap, OpParseContext> originalOpParseCtx = new LinkedHashMap, OpParseContext>(); + private LinkedHashMap, RowResolver> originalOpRowResolver = new LinkedHashMap, RowResolver>(); + private LinkedHashMap, Map> originalOpColumnExprMap = new LinkedHashMap, Map>(); + + private boolean isPhase1 = true; + + /** + * Transform the query tree. Firstly, find out correlations between operations. + * Then, group these operators in groups + * @param pactx + * current parse context + */ + public ParseContext transform(ParseContext pctx) throws SemanticException { + + if (isPhase1){ + + pGraphContext = pctx; + opParseCtx = pctx.getOpParseCtx(); + + YSmartNodePhase1ProcCtx ysmartCtx = new YSmartNodePhase1ProcCtx(); + + Map opRules = new LinkedHashMap(); + + Dispatcher disp = new DefaultRuleDispatcher(getPhase1DefaultProc(), opRules, ysmartCtx); + GraphWalker ogw = new DefaultGraphWalker(disp); + + // Create a list of topop nodes + ArrayList topNodes = new ArrayList(); + topNodes.addAll(pGraphContext.getTopOps().values()); + ogw.startWalking(topNodes, null); + isPhase1 = false; + + }else{ + + /* Types of correlations: + * 1) Input Correlation: Multiple nodes have input correlation + (IC) if their input relation sets are not disjoint; + 2) Transit Correlation: Multiple nodes have transit correlation + (TC) if they have not only input correlation, but + also the same partition key; + 3) Job Flow Correlation: A node has job flow correlation + (JFC) with one of its child nodes if it has the same + partition key as that child node. + * */ + /* Only IC+TC needs YSmartCompositeOperator, YSmartReduceSinkOperator and YSmartDispatchOperator. + * JFS only needs YSmartFakeReduceSinkOperator*/ + + + + pGraphContext = pctx; + opParseCtx = pctx.getOpParseCtx(); + initializeAliastoTabNameMapping(pGraphContext.getQB()); + System.out.println(AliastoTab.toString()); + //transformTPCHQ17ShowCase(); + + // 1: find out correlation + YSmartNodeProcCtx ysmartCtx = new YSmartNodeProcCtx(); + + Map opRules = new LinkedHashMap(); + opRules.put(new RuleRegExp("R1", "RS%"), new YSmartNodeProc()); + + Dispatcher disp = new DefaultRuleDispatcher(getDefaultProc(), opRules, ysmartCtx); + GraphWalker ogw = new DefaultGraphWalker(disp); + + // Create a list of topop nodes + ArrayList topNodes = new ArrayList(); + topNodes.addAll(pGraphContext.getTopOps().values()); + ogw.startWalking(topNodes, null); + + + // 2: transform the query plan tree + System.out.println("===========Begain Transforming====================================="); + for(YSmartCorrelation correlation: ysmartCtx.getCorrelations()){ + pGraphContext = YSmartOptimizerUtils.applyCorrelation(correlation, pGraphContext, originalOpColumnExprMap, originalOpRowResolver); + } + System.out.println("===========Transforming over====================================="); + + + System.out.println("===========Re-walk Tree start====================================="); + + ysmartCtx = new YSmartNodeProcCtx(); + + opRules = new LinkedHashMap(); + + disp = new DefaultRuleDispatcher(getDefaultProcForReWalk(), opRules, ysmartCtx); + ogw = new DefaultGraphWalker(disp); + + // Create a list of topop nodes + topNodes.clear(); + topNodes.addAll(pGraphContext.getTopOps().values()); + ogw.startWalking(topNodes, null); + System.out.println("===========Re-walk Tree over ====================================="); + + } + + return pGraphContext; + } + + + + private NodeProcessor getPhase1DefaultProc(){ + return new NodeProcessor() { + @Override + public Object process(Node nd, Stack stack, + NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { + Operator op = (Operator)nd; + OpParseContext opCtx= opParseCtx.get(op); + + + if(op.getColumnExprMap() != null){ + //System.out.println("put originalOpColumnExprMap:" + op.getColumnExprMap().toString()); + originalOpColumnExprMap.put(op, op.getColumnExprMap()); + } + //System.out.println("put originalOpParseCtx:" + opCtx.getRowResolver().getColumnInfos().toString()); + originalOpParseCtx.put(op, opCtx); + originalOpRowResolver.put(op, opCtx.getRowResolver()); + /* + System.out.println("=================================================="); + + if (op.getParentOperators() == null && op.getChildOperators() == null){ + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + null + + ", children: " + null); + }else if (op.getParentOperators() == null){ + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + null + + ", children: " + op.getChildOperators().toString()); + }else if (op.getChildOperators() == null){ + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + op.getParentOperators().toString() + + ", children: " + null); + }else{ + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + op.getParentOperators().toString() + + ", children: " + op.getChildOperators().toString()); + } + */ + + + return null; + } + }; + } + + private class YSmartNodeProc implements NodeProcessor { + + /** + * Find all peer ReduceSinkOperators (which have the same child operator of op) of op (op included). + */ + private ArrayList findPeerReduceSinkOperators(ReduceSinkOperator op){ + + ArrayList peerReduceSinkOperators = new ArrayList(); + + List> children = op.getChildOperators(); + assert children.size() == 1; + + for(Operator parent: children.get(0).getParentOperators()){ + assert (parent instanceof ReduceSinkOperator); + peerReduceSinkOperators.add((ReduceSinkOperator)parent); + } + + return peerReduceSinkOperators; + } + + private ArrayList findCorrelatedReduceSinkOperators(Operator op, + HashSet keyColumns) throws Exception{ + + + + ArrayList correlatedReduceSinkOps = new ArrayList(); + if(op.getParentOperators() == null){ + return correlatedReduceSinkOps; + } + if (originalOpColumnExprMap.get(op) == null && !(op instanceof ReduceSinkOperator)){ + assert op.getParentOperators().size() == 1; + correlatedReduceSinkOps.addAll(findCorrelatedReduceSinkOperators((Operator)op.getParentOperators().get(0), keyColumns)); + + //for (Object parent: op.getParentOperators()){ + // correlatedReduceSinkOps.addAll(findCorrelatedReduceSinkOperators((Operator)parent, keyColumns)); + //} + }else if(originalOpColumnExprMap.get(op) != null && !(op instanceof ReduceSinkOperator)){ + HashSet newKeyColumns = new HashSet(); + for (String keyColumn: keyColumns){ + ExprNodeDesc col = (ExprNodeDesc) originalOpColumnExprMap.get(op).get(keyColumn); + if(col instanceof ExprNodeColumnDesc){ + newKeyColumns.add(((ExprNodeColumnDesc)col).getColumn()); + } + } + System.out.println("newKeyColumns.toString():" + newKeyColumns.toString()); + + if(op.getName().equals("JOIN")){ + HashSet tableNeedToCheck = new HashSet(); + for (String keyColumn: keyColumns){ + for(ColumnInfo cinfo: originalOpParseCtx.get(op).getRowResolver().getColumnInfos()){ + System.out.println("keyColumn:" + keyColumn + " cinfo:" + cinfo.getInternalName()); + if(keyColumn.equals(cinfo.getInternalName())){ + tableNeedToCheck.add(cinfo.getTabAlias()); + + } + } + } + + System.out.println("There are " + tableNeedToCheck.size() + " which should be cheked for this Join OP"); + for (Object parent: op.getParentOperators()){ + assert originalOpParseCtx.get(parent).getRowResolver().getTableNames().size() == 1; + for(String tbl: originalOpParseCtx.get(parent).getRowResolver().getTableNames()){ + if(tableNeedToCheck.contains(tbl)){ + correlatedReduceSinkOps.addAll(findCorrelatedReduceSinkOperators((Operator)parent, newKeyColumns)); + break; + } + } + } + + }else{ + assert op.getParentOperators().size() == 1; + correlatedReduceSinkOps.addAll(findCorrelatedReduceSinkOperators((Operator)op.getParentOperators().get(0), newKeyColumns)); + } + + }else if(originalOpColumnExprMap.get(op) != null && op instanceof ReduceSinkOperator){ + + HashSet newKeyColumns = new HashSet(); + for (String keyColumn: keyColumns){ + ExprNodeDesc col = (ExprNodeDesc) originalOpColumnExprMap.get(op).get(keyColumn); + System.out.println(col.getName() + col.getCols().toString()); + if(col instanceof ExprNodeColumnDesc){ + newKeyColumns.add(((ExprNodeColumnDesc)col).getColumn()); + } + } + System.out.println(newKeyColumns.toString()); + ReduceSinkOperator rsop = (ReduceSinkOperator)op; + boolean isCorrelated = true; + System.out.println("rsop.getConf().getKeyCols() " + rsop.getConf().getKeyCols()); + for(ExprNodeDesc key: rsop.getConf().getKeyCols()){ + if (!(key instanceof ExprNodeColumnDesc )){ + isCorrelated = false; + } + if (!newKeyColumns.contains(((ExprNodeColumnDesc)key).getColumn())){ + isCorrelated = false; + } + } + if (isCorrelated){ + if (((Operator)(op.getChildOperators().get(0))).getName().equals("JOIN")){ + ArrayList peers = findPeerReduceSinkOperators(rsop); + correlatedReduceSinkOps.addAll(peers); + }else{ + correlatedReduceSinkOps.add(rsop); + } + }else{ + correlatedReduceSinkOps.clear(); + } + + + + }else{ + throw new Exception("YSmart optimizer: ReduceSinkOperator " + op.getIdentifier() + " does not have ColumnExprMap"); + } + + + + return correlatedReduceSinkOps; + + } + + + private ArrayList exploitJFC(ReduceSinkOperator op, YSmartNodeProcCtx ysmartCtx, YSmartCorrelation correlation){ + + ysmartCtx.addWalked(op); + correlation.addToAllReduceSinkOperators(op); + + ArrayList ReduceSinkOperators = new ArrayList(); + + boolean isStop = false; + boolean exploit = true; + + ArrayList keys = op.getConf().getKeyCols(); + HashSet keyColumns = new HashSet(); + for(ExprNodeDesc key: keys){ + if (!(key instanceof ExprNodeColumnDesc)){ + exploit = false; + }else{ + keyColumns.add(((ExprNodeColumnDesc)key).getColumn()); + } + } + + + + if (exploit){ + System.out.println("exploit:" + exploit + ", " + op.getIdentifier()); + System.out.println("exploit:" + keyColumns.toString()); + ArrayList newReduceSinkOperators = new ArrayList(); + for (Operator parent: op.getParentOperators()){ + try { + ArrayList correlatedReduceSinkOperators = + findCorrelatedReduceSinkOperators(parent, keyColumns); + if (correlatedReduceSinkOperators == null || correlatedReduceSinkOperators.size() == 0){ + //ReduceSinkOperators.add(op); + newReduceSinkOperators.add(op); + }else{ + for(ReduceSinkOperator rsop: correlatedReduceSinkOperators){ + + + if ( !correlation.getUp2downRSops().containsKey(op) ){ + correlation.getUp2downRSops().put(op, new ArrayList()); + } + correlation.getUp2downRSops().get(op).add(rsop); + + if ( !correlation.getDown2upRSops().containsKey(rsop)){ + correlation.getDown2upRSops().put(rsop, new ArrayList()); + } + correlation.getDown2upRSops().get(rsop).add(op); + + ArrayList exploited = exploitJFC(rsop, ysmartCtx, correlation); + if (exploited == null || exploited.size() == 0){ + newReduceSinkOperators.add(rsop); + }else{ + newReduceSinkOperators.addAll(exploited); + } + + } + } + + } catch (Exception e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } + + //List> ops = op.getChildOperators(); + + //while(isStop){ + + //} + ReduceSinkOperators.clear(); + ReduceSinkOperators.addAll(newReduceSinkOperators); + } + + + return ReduceSinkOperators; + } + + + private TableScanOperator findTableScanOPerator(Operator startPoint){ + Operator thisOp = (Operator) startPoint.getParentOperators().get(0); + while(true){ + if(thisOp.getName().equals("RS")) { + return null; + }else if(thisOp.getName().equals("TS")){ + return (TableScanOperator)thisOp; + } + else{ + if (thisOp.getParentOperators() != null) { + thisOp = (Operator) thisOp.getParentOperators().get(0); + } + else{ + break; + } + } + } + return null; + } + + private void annotateOpPlan(YSmartCorrelation correlation){ + HashMap bottomReduceSink2OpPlanMap = new HashMap(); + int count = 0; + for(ReduceSinkOperator rsop: correlation.getBottomReduceSinkOperators()){ + if (!bottomReduceSink2OpPlanMap.containsKey(rsop)){ + bottomReduceSink2OpPlanMap.put(rsop, count); + for(ReduceSinkOperator peerRSop: findPeerReduceSinkOperators(rsop)){ + bottomReduceSink2OpPlanMap.put(peerRSop, count); + } + count++; + } + } + correlation.setBottomReduceSink2OpPlanMap(bottomReduceSink2OpPlanMap); + + } + + + public Object process(Node nd, Stack stack, NodeProcessorCtx ctx, + Object... nodeOutputs) throws SemanticException { + + YSmartNodeProcCtx ysmartCtx = (YSmartNodeProcCtx)ctx; + + ReduceSinkOperator op = (ReduceSinkOperator)nd; + + + + + System.out.println("=================================================="); + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName()); + if (op.getColumnExprMap()!=null){ + System.out.println("originalOpColumnExprMap: " + originalOpColumnExprMap.get(op).toString()); + } + if(originalOpParseCtx.get(op) != null){ + System.out.println(originalOpParseCtx.get(op).getRowResolver() == null); + System.out.println(originalOpParseCtx.get(op).getRowResolver().getColumnInfos() == null); + System.out.println(opParseCtx.get(op).getRowResolver() == null); + System.out.println(opParseCtx.get(op).getRowResolver().getColumnInfos() == null); + System.out.println("originalOpParseCtx: " + originalOpParseCtx.get(op).getRowResolver().getColumnInfos().toString()); + } + if(originalOpRowResolver.get(op) != null){ + System.out.println("originalOpRowResolver: " + originalOpRowResolver.get(op).getColumnInfos().toString()); + } + + OpParseContext opCtx= opParseCtx.get(op); + System.out.println(opCtx.getRowResolver().getTableNames().toString()); + for(ColumnInfo cinfo: opCtx.getRowResolver().getColumnInfos()){ + System.out.println(cinfo.getInternalName() + ", " + cinfo.getAlias() + ", " + + cinfo.getTabAlias() + ", " + cinfo.getType().toString()); + } + String tableName = opCtx.getRowResolver().getTableNames().toString(); + + + System.out.println(op.getConf().getKeyCols().toString()); + String parents = "", children = ""; + for(Operator parent: op.getParentOperators()){ + parents += "(" + parent.getIdentifier() + ", " + parent.getName() + ")" + ", "; + } + for(Operator child: op.getChildOperators()){ + children += "(" + child.getIdentifier() + ", " +child.getName() + ")" + ", "; + } + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + parents + + ", children: " + children); + + + if (ysmartCtx.isWalked(op)){ + return null; + } + + if (op.getConf().getKeyCols().size() == 0){ + ysmartCtx.addWalked(op); + return null; + } + + // 1: find out correlation + YSmartCorrelation correlation = new YSmartCorrelation(); + ArrayList peerReduceSinkOperators = findPeerReduceSinkOperators(op); + ArrayList bottomReduceSinkOperators = new ArrayList(); + for(ReduceSinkOperator rsop: peerReduceSinkOperators){ + + ArrayList thisBottomReduceSinkOperators= exploitJFC(rsop, ysmartCtx, correlation); + if (thisBottomReduceSinkOperators.size() == 0){ + thisBottomReduceSinkOperators.add(rsop); + }else{ + boolean isClear = false; + for(ReduceSinkOperator bottomrsop: thisBottomReduceSinkOperators){ + TableScanOperator tsop = findTableScanOPerator(bottomrsop); + if (tsop == null){ + isClear = true; // currently we only handle correlations involving source tables + }else{ + if ( !correlation.getTop2TSops().containsKey(rsop) ){ + correlation.getTop2TSops().put(rsop, new ArrayList()); + } + correlation.getTop2TSops().get(rsop).add(tsop); + + if ( !correlation.getBottom2TSops().containsKey(bottomrsop)){ + correlation.getBottom2TSops().put(bottomrsop, new ArrayList()); + } + correlation.getBottom2TSops().get(bottomrsop).add(tsop); + } + } + if(isClear){ + thisBottomReduceSinkOperators.clear(); + thisBottomReduceSinkOperators.add(rsop); + } + + } + bottomReduceSinkOperators.addAll(thisBottomReduceSinkOperators); + } + + + + if(!peerReduceSinkOperators.containsAll(bottomReduceSinkOperators)){ + correlation.setJobFlowCorrelation(true); + correlation.setJFCCorrelation(peerReduceSinkOperators, bottomReduceSinkOperators); + annotateOpPlan(correlation); + } + + + + if (correlation.hasJobFlowCorrelation()){ + boolean hasICandTC = findICandTC(correlation); + correlation.setInputCorrelation(hasICandTC); + correlation.setTransitCorrelation(hasICandTC); + ysmartCtx.addCorrelation(correlation); + + for(ReduceSinkOperator rsop: correlation.getTopReduceSinkOperators()){ + System.out.println(rsop.getIdentifier() + ", " + rsop.getConf().getKeyCols().toString()); + } + System.out.println(">>>>>>>>>>>>>>>>>>>>"); + for(ReduceSinkOperator rsop: correlation.getBottomReduceSinkOperators()){ + System.out.println(rsop.getIdentifier() + ", " + rsop.getConf().getKeyCols().toString()); + } + + System.out.println("===correlation.getUp2downRSops()==="); + for(Entry> entry: correlation.getUp2downRSops().entrySet()){ + for(ReduceSinkOperator rsop: entry.getValue()){ + System.out.println("up: " + entry.getKey().getIdentifier() + ", " + entry.getKey().getConf().getKeyCols().toString() + + ";down: " + rsop.getIdentifier() + ", " + rsop.getConf().getKeyCols().toString()); + } + } + + System.out.println("===correlation.getDown2upRSops()==="); + for(Entry> entry: correlation.getDown2upRSops().entrySet()){ + for(ReduceSinkOperator rsop: entry.getValue()){ + System.out.println("up: " + rsop.getIdentifier() + ", " + rsop.getConf().getKeyCols().toString() + + ";down: " + entry.getKey().getIdentifier() + ", " + entry.getKey().getConf().getKeyCols().toString()); + } + } + + System.out.println("===correlation.getTop2TSops()==="); + for(Entry> entry: correlation.getTop2TSops().entrySet()){ + for(TableScanOperator tsop: entry.getValue()){ + System.out.println("top rs: " + entry.getKey().getIdentifier() + ", " + entry.getKey().getConf().getKeyCols().toString() + + ";ts: " + tsop.getIdentifier()); + } + } + + System.out.println("===correlation.getBottom2TSops==="); + for(Entry> entry: correlation.getBottom2TSops().entrySet()){ + for(TableScanOperator tsop: entry.getValue()){ + System.out.println("bottom rs: " + entry.getKey().getIdentifier() + ", " + entry.getKey().getConf().getKeyCols().toString() + + ";ts: " + tsop.getIdentifier()); + } + } + + if (hasICandTC){ + System.out.println("===table 2 correlated RSops==="); + for(Entry> entry: correlation.getTable2CorrelatedRSops().entrySet()){ + for(ReduceSinkOperator rsop: entry.getValue()){ + System.out.println("table: " + entry.getKey() + ", " + rsop.getIdentifier() + " " + rsop.getName()); + } + } + + System.out.println("===table 2 correlated TSops==="); + for(Entry> entry: correlation.getTable2CorrelatedTSops().entrySet()){ + for(TableScanOperator rsop: entry.getValue()){ + System.out.println("table: " + entry.getKey() + ", " + rsop.getIdentifier() + " " + rsop.getName()); + } + } + } + + System.out.println("===bottomReducesink2Opplan==="); + for(Entry entry: correlation.getBottomReduceSink2OpPlanMap().entrySet()){ + System.out.println("rsop: " + entry.getKey().getIdentifier() + ", opPlanTag:" + entry.getValue()); + } + + } + + ysmartCtx.addWalked(op); + + return null; + } + + private boolean findICandTC(YSmartCorrelation correlation){ + + boolean hasICandTC = false; + HashMap> table2RSops = new HashMap>(); + HashMap> table2TSops = new HashMap>(); + + for (Entry> entry: correlation.getBottom2TSops().entrySet()){ + //System.out.println("table name: " + AliastoTabName.get( + // entry.getValue().get(0).getConf().getAlias())); + String tbl = AliastoTabName.get(entry.getValue().get(0).getConf().getAlias()); + if (!table2RSops.containsKey(tbl) && !table2TSops.containsKey(tbl)){ + table2RSops.put(tbl, new ArrayList()); + table2TSops.put(tbl, new ArrayList()); + } + assert entry.getValue().size() == 1; + table2RSops.get(tbl).add(entry.getKey()); + table2TSops.get(tbl).add(entry.getValue().get(0)); + } + /* + for (String tbl: table2RSops.keySet()){ + if(table2RSops.get(tbl).size() < 2){ + table2RSops.remove(tbl); + table2TSops.remove(tbl); + } + } + */ + + if (table2RSops.size() > 0){ + hasICandTC = true; + correlation.setICandTCCorrelation(table2RSops, table2TSops); + } + + /* + + for (Entry> entry1: correlation.getBottom2TSops().entrySet()){ + for (Entry> entry2: correlation.getBottom2TSops().entrySet()){ + if (entry1 != entry2){ + System.out.println("table name1: " + AliastoTabName.get( + entry1.getValue().get(0).getConf().getAlias()) + ", table name2: " + AliastoTabName.get( + entry2.getValue().get(0).getConf().getAlias())); + String tbl1 = AliastoTabName.get(entry1.getValue().get(0).getConf().getAlias()); + String tbl2 = AliastoTabName.get(entry1.getValue().get(0).getConf().getAlias()); + if (AliastoTabName.get(entry1.getValue().get(0).getConf().getAlias()).equals + (AliastoTabName.get(entry2.getValue().get(0).getConf().getAlias()))){ + hasICandTC = true; + if (table2RSops.containsKey(key)) + + } + } + } + + } + + + */ + + return hasICandTC; + } + + + + } + + + private NodeProcessor getDefaultProcForReWalk() { + + return new NodeProcessor() { + @Override + public Object process(Node nd, Stack stack, + NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { + Operator op = (Operator)nd; + + + + System.out.println("=================================================="); + + if (op.getColumnExprMap()!=null){ + System.out.println("op.getColumnExprMap(): " + op.getColumnExprMap().toString()); + } + if(pGraphContext.getOpParseCtx().get(op) != null){ + System.out.println("pGraphContext.getOpParseCtx().get(op): " + pGraphContext.getOpParseCtx().get(op).getRowResolver().getColumnInfos().toString()); + } + + OpParseContext opCtx= opParseCtx.get(op); + System.out.println(opCtx.getRowResolver().getTableNames().toString()); + for(ColumnInfo cinfo: opCtx.getRowResolver().getColumnInfos()){ + System.out.println(cinfo.getInternalName() + ", " + cinfo.getAlias() + ", " + + cinfo.getTabAlias() + ", " + cinfo.getType().toString()); + } + + + if (op.getParentOperators() == null && op.getChildOperators() == null){ + + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + null + + ", children: " + null); + }else if (op.getParentOperators() == null){ + String parents = "", children = ""; + + for(Operator child: op.getChildOperators()){ + children += "(" + child.getIdentifier() + ", " +child.getName() + ")" + ", "; + } + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + null + + ", children: " + children); + + + }else if (op.getChildOperators() == null){ + + String parents = "", children = ""; + for(Operator parent: op.getParentOperators()){ + parents += "(" + parent.getIdentifier() + ", " + parent.getName() + ")" + ", "; + } + + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + parents + + ", children: " + null); + + }else{ + String parents = "", children = ""; + for(Operator parent: op.getParentOperators()){ + parents += "(" + parent.getIdentifier() + ", " + parent.getName() + ")" + ", "; + } + for(Operator child: op.getChildOperators()){ + children += "(" + child.getIdentifier() + ", " +child.getName() + ")" + ", "; + } + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + parents + + ", children: " + children); + + } + + + + return null; + } + }; + } + + + + private NodeProcessor getDefaultProc() { + return new NodeProcessor() { + @Override + public Object process(Node nd, Stack stack, + NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { + Operator op = (Operator)nd; + + + + System.out.println("=================================================="); + + if (op.getColumnExprMap()!=null){ + System.out.println("originalOpColumnExprMap: " + originalOpColumnExprMap.get(op).toString()); + } + if(originalOpParseCtx.get(op) != null){ + System.out.println("originalOpParseCtx: " + originalOpParseCtx.get(op).getRowResolver().getColumnInfos().toString()); + } + if(originalOpRowResolver.get(op) != null){ + System.out.println("originalOpRowResolver: " + originalOpRowResolver.get(op).getColumnInfos().toString()); + } + + + OpParseContext opCtx= opParseCtx.get(op); + System.out.println(opCtx.getRowResolver().getTableNames().toString()); + for(ColumnInfo cinfo: opCtx.getRowResolver().getColumnInfos()){ + System.out.println(cinfo.getInternalName() + ", " + cinfo.getAlias() + ", " + + cinfo.getTabAlias() + ", " + cinfo.getType().toString()); + } + + + if (op.getParentOperators() == null && op.getChildOperators() == null){ + + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + null + + ", children: " + null); + }else if (op.getParentOperators() == null){ + String parents = "", children = ""; + + for(Operator child: op.getChildOperators()){ + children += "(" + child.getIdentifier() + ", " +child.getName() + ")" + ", "; + } + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + null + + ", children: " + children); + + + }else if (op.getChildOperators() == null){ + + String parents = "", children = ""; + for(Operator parent: op.getParentOperators()){ + parents += "(" + parent.getIdentifier() + ", " + parent.getName() + ")" + ", "; + } + + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + parents + + ", children: " + null); + + }else{ + String parents = "", children = ""; + for(Operator parent: op.getParentOperators()){ + parents += "(" + parent.getIdentifier() + ", " + parent.getName() + ")" + ", "; + } + for(Operator child: op.getChildOperators()){ + children += "(" + child.getIdentifier() + ", " +child.getName() + ")" + ", "; + } + System.out.println("id: " + op.getIdentifier() + + ", name: " + op.getName() + + ", parents: " + parents + + ", children: " + children); + + } + + + + return null; + } + }; + } + + private class YSmartNodePhase1ProcCtx implements NodeProcessorCtx { + + } + + + public class YSmartCorrelation{ + + //private HashMap, ArrayList> correlations = new HashMap, ArrayList>(); + + private HashMap> down2upRSops = new HashMap>(); + private HashMap> up2downRSops = new HashMap>(); + + private HashMap> top2TSops = new HashMap>(); + private HashMap> bottom2TSops = new HashMap>(); + + private ArrayList topReduceSinkOperators; + private ArrayList bottomReduceSinkOperators; + + private HashMap> table2RSops; + private HashMap> table2TSops; + + private HashMap bottomReduceSink2OpPlanMap; + + private HashMap> dispatchConf = new HashMap>(); //inputTag->(Child->outputTag) + private HashMap> dispatchValueSelectDescConf = new HashMap>(); //inputTag->(Child->SelectDesc) + private HashMap> dispatchKeySelectDescConf = new HashMap>(); //inputTag->(Child->SelectDesc) + + private HashSet allReduceSinkOperators = new HashSet(); + + public void addToAllReduceSinkOperators(ReduceSinkOperator rsop){ + allReduceSinkOperators.add(rsop); + } + + public HashSet getAllReduceSinkOperators(){ + return allReduceSinkOperators; + } + + public HashMap> getDispatchConf(){ + return dispatchConf; + } + + public HashMap> getDispatchValueSelectDescConf(){ + return dispatchValueSelectDescConf; + } + + public HashMap> getDispatchKeySelectDescConf(){ + return dispatchKeySelectDescConf; + } + + public void addOpPlanToDispatchConf(Integer opPlan){ + if(!dispatchConf.containsKey(opPlan)) { + dispatchConf.put(opPlan, new HashMap()); + } + } + + public HashMap getDispatchConfForOpPlan(Integer opPlan){ + return dispatchConf.get(opPlan); + } + + public void addOpPlanToDispatchValueSelectDescConf(Integer opPlan){ + if(!dispatchValueSelectDescConf.containsKey(opPlan)) { + dispatchValueSelectDescConf.put(opPlan, new HashMap()); + } + } + + public HashMap getDispatchValueSelectDescConfForOpPlan(Integer opPlan){ + return dispatchValueSelectDescConf.get(opPlan); + } + + public void addOpPlanToDispatchKeySelectDescConf(Integer opPlan){ + if(!dispatchKeySelectDescConf.containsKey(opPlan)) { + dispatchKeySelectDescConf.put(opPlan, new HashMap()); + } + } + + public HashMap getDispatchKeySelectDescConfForOpPlan(Integer opPlan){ + return dispatchKeySelectDescConf.get(opPlan); + } + + + private boolean inputCorrelation = false; + private boolean transitCorrelation = false; + private boolean jobFlowCorrelation = false; + + public void setBottomReduceSink2OpPlanMap(HashMap bottomReduceSink2OpPlanMap){ + this.bottomReduceSink2OpPlanMap = bottomReduceSink2OpPlanMap; + } + + public HashMap getBottomReduceSink2OpPlanMap(){ + return bottomReduceSink2OpPlanMap; + } + + public void setInputCorrelation(boolean inputCorrelation){ + this.inputCorrelation = inputCorrelation; + } + + public boolean hasInputCorrelation(){ + return inputCorrelation; + } + + public void setTransitCorrelation(boolean transitCorrelation){ + this.transitCorrelation = transitCorrelation; + } + + public boolean hasTransitCorrelation(){ + return transitCorrelation; + } + + public void setJobFlowCorrelation(boolean jobFlowCorrelation){ + this.jobFlowCorrelation = jobFlowCorrelation; + } + + public boolean hasJobFlowCorrelation(){ + return jobFlowCorrelation; + } + + + + public HashMap> getTop2TSops(){ + return top2TSops; + } + + public HashMap> getBottom2TSops(){ + return bottom2TSops; + } + + public HashMap> getDown2upRSops(){ + return down2upRSops; + } + + public HashMap> getUp2downRSops(){ + return up2downRSops; + } + + public void setJFCCorrelation(ArrayList peerReduceSinkOperators, + ArrayList bottomReduceSinkOperators){ + this.topReduceSinkOperators = peerReduceSinkOperators; + this.bottomReduceSinkOperators = bottomReduceSinkOperators; + } + + + public ArrayList getTopReduceSinkOperators(){ + return topReduceSinkOperators; + } + + public ArrayList getBottomReduceSinkOperators(){ + return bottomReduceSinkOperators; + } + + public void setICandTCCorrelation(HashMap> table2RSops, + HashMap> table2TSops){ + this.table2RSops = table2RSops; + this.table2TSops = table2TSops; + } + + public HashMap> getTable2CorrelatedRSops(){ + return table2RSops; + } + + public HashMap> getTable2CorrelatedTSops(){ + return table2TSops; + } + + + + } + + private class YSmartNodeProcCtx implements NodeProcessorCtx { + + private HashSet walked = new HashSet(); + + private ArrayList correlations = new ArrayList(); + + /*private HashMap, ArrayList> correlations = new HashMap, ArrayList>(); + + private HashMap> down2upRSops = new HashMap>(); + private HashMap> up2downRSops = new HashMap>(); + + private HashMap> top2TSops = new HashMap>(); + private HashMap> bottom2TSops = new HashMap>(); + + public HashMap> getTop2TSops(){ + return top2TSops; + } + + public HashMap> getBottom2TSops(){ + return bottom2TSops; + } + + public HashMap> getDown2upRSops(){ + return down2upRSops; + } + + public HashMap> getUp2downRSops(){ + return up2downRSops; + } + + public void addCorrelations(ArrayList peerReduceSinkOperators, + ArrayList bottomReduceSinkOperators){ + correlations.put(peerReduceSinkOperators , bottomReduceSinkOperators); + } + + public HashMap, ArrayList> getCorrelations(){ + return correlations; + } + */ + + public void addCorrelation(YSmartCorrelation correlation){ + correlations.add(correlation); + } + + public ArrayList getCorrelations(){ + return correlations; + } + + + public boolean isWalked(ReduceSinkOperator op){ + return walked.contains(op); + } + + public void addWalked(ReduceSinkOperator op){ + walked.add(op); + } + + public void addAllWalked(List ops){ + walked.addAll(ops); + } + + } + + +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartCompositeOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartCompositeOperator.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartCompositeOperator.java (revision 0) @@ -0,0 +1,176 @@ +package org.apache.hadoop.hive.ql.exec; + +import java.io.Serializable; +import java.util.ArrayList; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.YSmartCompositeDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; +import org.apache.hadoop.io.LongWritable; + +public class YSmartCompositeOperator extends Operator implements Serializable{ + + static final private Log LOG = LogFactory.getLog(Driver.class.getName()); + static final private LogHelper console = new LogHelper(LOG); + + public static enum Counter { + FILTERED, PASSED + } + + /** + * + */ + private static final long serialVersionUID = 1L; + + //Map the input alias to correspoding operators at the top of map phase + //private HashMap> AliasToTopMapOperators; + //Map the operator at the top of map phase to the corresponding operation, e.g. Join or group by + //private HashMap> TopOperatorToOperations; + //Map the operation to the operator at the top of reduce phase + //private HashMap OperationToTopReduceOperator; + + private ArrayList> topInternalOps; + private ArrayList ymfOps; + + private YSmartReduceSinkOperator correspondingYRS; + + private transient final LongWritable filtered_count, passed_count; + + + public YSmartCompositeOperator(){ + super(); + //topInternalOps = new ArrayList>(); + //ymfOps = new ArrayList(); + filtered_count = new LongWritable(); + passed_count = new LongWritable(); + } + + @Override + public OperatorType getType() { + return OperatorType.YSMARTCOMPOSITE; + } + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return new String("YCO"); + } + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + correspondingYRS = conf.getCorrespondingYRS(); + topInternalOps = conf.getTopInternalOps(); + ymfOps = conf.getYmfOps(); + allOpTags = conf.getAllOpTags(); + statsMap.put(Counter.FILTERED, filtered_count); + statsMap.put(Counter.PASSED, passed_count); + //TODO: Initialize internal operators + // I am not sure what I have done below is fine. + for (Operator op:topInternalOps){ + op.initialize(hconf, inputObjInspectors); + } + //initialize its internal operators first + + //initialize its children + initializeChildren(hconf); + } + + private int[] allOpTags; + + /* + public void setInternalNodes(ArrayList> topInternalOps, + ArrayList> bottomInternalOps, int[] allOpTags){ + this.allOpTags = allOpTags; + // the size of topInternalOps and bottomInternalOps should be same. + this.topInternalOps.addAll(topInternalOps); + //List> childrenOfThisOp = new ArrayList>(); + for (Operator op: bottomInternalOps){ + //List> originalChilren = op.getChildOperators(); + YSmartManualForwardOperator ymf = new YSmartManualForwardOperator(); + ymf.setChildOperators(null); + ymfOps.add(ymf); + List> newChildren = + new ArrayList>(); + newChildren.add(ymf); + op.setChildOperators(newChildren); + ymf.setParentOperators(Utilities.makeList(op)); + + } + + + }*/ + + + + @Override + public void processOp(Object row, int tag) throws HiveException { + //System.out.println("in YCO row:" + row.toString()); + ArrayList opTags = new ArrayList(); + boolean isForward = false; + for (Operator op: topInternalOps){ + //LOG.info("YCO: forwarding to " + op.getName()); + op.process(row, tag); + } + ArrayList rows = new ArrayList(); + int[] tags = new int[topInternalOps.size()]; + int i = 0; + for (YSmartManualForwardOperator ymf: ymfOps){ + rows.add(ymf.getRow()); + tags[i] = ymf.getTag(); + i++; + } + if (rows.size() > 0){ + i = 0; + + for (Object r: rows){ + if (r != null){ + opTags.add(allOpTags[i]); + isForward = true; + } + /* + if (r == null){ + + if (row instanceof org.apache.hadoop.hive.serde2.lazy.LazyStruct){ + ((org.apache.hadoop.hive.serde2.lazy.LazyStruct)row).opNotTags.add(i); + }else{ + ((org.apache.hadoop.hive.serde2.lazy.LazyStruct)((Object[])row)[0]).opNotTags.add(i); + } + } + else{ + isForward = true; + } + */ + i++; + } + if (correspondingYRS != null){ + //System.out.println("set opTags to correspondingYRS " + + // correspondingYRS.getIdentifier() + " " + opTags.toString()); + correspondingYRS.setOpTags(opTags); + }else{ + console.printInfo("opTags.toString:" + opTags.toString()); + } + + + } + if (isForward){ + passed_count.set(passed_count.get() + 1); + forward(row, inputObjInspectors[tag]); + //System.out.println("Forward row:" + row.toString() + "|opTags:" + opTags.toString() + "|tag:" + tag); + } + else{ + filtered_count.set(filtered_count.get() + 1); + } + for (YSmartManualForwardOperator ymf: ymfOps){ + ymf.clean(); + } + + } + +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartForwardOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartForwardOperator.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartForwardOperator.java (revision 0) @@ -0,0 +1,179 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.io.Serializable; +import java.util.ArrayList; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.YSmartForwardDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.io.BytesWritable; + +/** + * YSmart Forward Operator is the operator for opPlan of which the bottomRSOP and + * topRSOP is the same one. So we need to add YSmart forward op to the original RS and Join or Groupby, + * so this forwarder will connect to YDP. + **/ +public class YSmartForwardOperator extends Operator implements + Serializable { + private static final long serialVersionUID = 1L; + + private int realOutputInspectorIndicator; + + private ArrayList storage; + private int counter = 0; + private int counterActualForwarded = 0; + + + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + realOutputInspectorIndicator = conf.getRealOutputInspectorIndicator(); + outputObjInspector = inputObjInspectors[realOutputInspectorIndicator]; + System.out.println("YSmart Forwarder outputObjInspector: " + + this.getChildOperators().get(0).getParentOperators().indexOf(this) + + " " + outputObjInspector.getTypeName()); + for (Operator op : childOperators) { + //System.out.println("YSmart forward children: " + op.getIdentifier() + " " + op.getName()); + } + childOperatorsTag[0] = realOutputInspectorIndicator; + storage = new ArrayList(); + + + + initializeChildren(hconf); + } + + + + + private Object keyObject; + + //private ArrayList bufferedRow = new ArrayList(3); + + + private BytesWritable groupKey; //Yin + + @Override + public void processOp(Object row, int tag) throws HiveException { + + + //ArrayList bufferedRow = new ArrayList(3); + + ////System.out.println("((Object[])((ArrayList)row).get(0))[0].toString(): " + ((Object[])((ArrayList)row).get(0))[0].toString()); + ////System.out.println("((Object[])((ArrayList)row).get(1))[0].toString(): " + ((Object[])((ArrayList)row).get(1))[0].toString()); + if (!bytesWeitableGroupKey.equals(groupKey)) { + + if(groupKey == null){ + groupKey = new BytesWritable(); + + }else{ + if(!groupKey.equals(childOperatorsArray[0].getBytesWritableGroupKey())){ + //System.out.println("forward: processOp update key " + groupKey.toString()); + //System.out.println("YFOP: end & startGroup() in processOP"); + childOperatorsArray[0].endGroup(); + childOperatorsArray[0].startGroup(); + keyObject = ((ArrayList)row).get(0); + childOperatorsArray[0].setGroupKeyObject(keyObject); + childOperatorsArray[0].setBytesWritableGroupKey(groupKey); + } + //System.out.println("forward: processOp storage.size:" + storage.size() + " bytesWeitableGroupKey:" + bytesWeitableGroupKey.toString() + + // " groupKey: " +groupKey ); + forwardAllAndClearBuffer(); + } + groupKey.set(bytesWeitableGroupKey.get(), 0, bytesWeitableGroupKey.getSize()); + } + + counter++; + //System.out.println("add into buffer: " + row.toString()); + //bufferedRow.addAll((List)row); + storage.add(row); + //System.out.println("forward: ((Object[])((ArrayList)storage.get(0)).get(0))[0].toString(): " + ((Object[])((ArrayList)storage.get(0)).get(0))[0].toString()); + //System.out.println("forward: ((Object[])((ArrayList)storage.get(0)).get(1))[0].toString(): " + ((Object[])((ArrayList)storage.get(0)).get(1))[0].toString()); + //System.out.println("forward: ((Object[])((ArrayList)storage.get(0)).get(1))[0].toString(): " + ((Object[])((ArrayList)storage.get(0)).get(1))[1].toString()); + ////System.out.println("((Object[])((ArrayList)row).get(0))[0].toString(): " + ((Object[])((List)bufferedRow).get(0))[0].toString()); + ////System.out.println("((Object[])((ArrayList)row).get(1))[0].toString(): " + ((Object[])((List)bufferedRow).get(1))[0].toString() + " " + + // ((Object[])((List)bufferedRow).get(1))[1].toString()); + //forward(row, inputObjInspectors[realOutputInspectorIndicator]); + + + } + + private void forwardAllAndClearBuffer() throws HiveException{ + ////System.out.println(this.getIdentifier() + " ysmart forwarder end group forwardAll, buffer size " + storage.size());//Yin + for(Object row: storage){ + counterActualForwarded++; + //System.out.println("forward: " + row.toString()); + //System.out.println("forward: ((Object[])((ArrayList)row).get(0))[0].toString(): " + ((Object[])((ArrayList)row).get(0))[0].toString()); + //System.out.println("forward: ((Object[])((ArrayList)row).get(1))[0].toString(): " + ((Object[])((ArrayList)row).get(1))[0].toString()); + //forward(row, inputObjInspectors[realOutputInspectorIndicator]); + forward(row, outputObjInspector); + + } + storage.clear(); + } + + @Override + public void closeOp(boolean abort) throws HiveException { + if(!abort){ + if(!groupKey.equals(childOperatorsArray[0].getBytesWritableGroupKey())){ + ////System.out.println("forward: closeOp update key"); + //System.out.println("YFOP: startGroup() in closeOp"); + childOperatorsArray[0].endGroup(); + childOperatorsArray[0].startGroup(); + childOperatorsArray[0].setGroupKeyObject(keyObject); + childOperatorsArray[0].setBytesWritableGroupKey(groupKey); + } + ////System.out.println("forward: closeOp storage.size:" + storage.size()); + //System.out.println("YFOP: getNumOfParentEndOP()" + childOperatorsArray[0].getNumOfParentEndOP() + " counter" + counter + " actualCounter:" + counterActualForwarded); + forwardAllAndClearBuffer(); + //System.out.println("YFOP: getNumOfParentEndOP()" + childOperatorsArray[0].getNumOfParentEndOP() + " counter" + counter + " actualCounter:" + counterActualForwarded); + if(childOperatorsArray[0].getNumOfParentEndOP() == childOperatorsArray[0].getParentOperators().size() - 1){ + childOperatorsArray[0].endGroup(); + } + } + } + + @Override + public void startGroup() throws HiveException { + + + + } + + @Override + public void endGroup() throws HiveException { + + + + } + + @Override + public void setGroupKeyObject(Object keyObject) { + + } + + + @Override + public OperatorType getType() { + return OperatorType.YSMARTFORWARD; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartFakeReduceSinkOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartFakeReduceSinkOperator.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartFakeReduceSinkOperator.java (revision 0) @@ -0,0 +1,590 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.io.HiveKey; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.plan.YSmartFakeReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.Serializer; +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspector.StandardUnion; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * Reduce Sink Operator sends output to the reduce stage. + **/ +public class YSmartFakeReduceSinkOperator extends Operator + implements Serializable { + + private static final long serialVersionUID = 1L; + + /** + * The evaluators for the key columns. Key columns decide the sort order on + * the reducer side. Key columns are passed to the reducer in the "key". + */ + protected transient ExprNodeEvaluator[] keyEval; + /** + * The evaluators for the value columns. Value columns are passed to reducer + * in the "value". + */ + protected transient ExprNodeEvaluator[] valueEval; + /** + * The evaluators for the partition columns (CLUSTER BY or DISTRIBUTE BY in + * Hive language). Partition columns decide the reducer that the current row + * goes to. Partition columns are not passed to reducer. + */ + //protected transient ExprNodeEvaluator[] partitionEval; + + // TODO: we use MetadataTypedColumnsetSerDe for now, till DynamicSerDe is + // ready + transient Serializer keySerializer; + transient boolean keyIsText; + transient Serializer valueSerializer; + + transient TableDesc keyTableDesc; + transient TableDesc valueTableDesc; + + transient Deserializer inputKeyDeserializer; + + transient SerDe inputValueDeserializer; + + transient int tag; + transient byte[] tagByte = new byte[1]; + transient ByteWritable tagWritable = new ByteWritable(); + transient protected int numDistributionKeys; + transient protected int numDistinctExprs; + + private static String[] fieldNames; + + static { + ArrayList fieldNameArray = new ArrayList(); + for (Utilities.ReduceField r : Utilities.ReduceField.values()) { + fieldNameArray.add(r.toString()); + } + fieldNames = fieldNameArray.toArray(new String[0]); + } + + private ArrayList peers; + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + + System.out.println("Yin DEBUG " + id + " " + this.getName() + " " + this.toString()); + assert childOperatorsArray.length == 1; + try { + keyEval = new ExprNodeEvaluator[conf.getKeyCols().size()]; + int i = 0; + for (ExprNodeDesc e : conf.getKeyCols()) { + keyEval[i++] = ExprNodeEvaluatorFactory.get(e); + } + + numDistributionKeys = conf.getNumDistributionKeys(); + distinctColIndices = conf.getDistinctColumnIndices(); + numDistinctExprs = distinctColIndices.size(); + + valueEval = new ExprNodeEvaluator[conf.getValueCols().size()]; + i = 0; + for (ExprNodeDesc e : conf.getValueCols()) { + valueEval[i++] = ExprNodeEvaluatorFactory.get(e); + } + + //partitionEval = new ExprNodeEvaluator[conf.getPartitionCols().size()]; + //i = 0; + //for (ExprNodeDesc e : conf.getPartitionCols()) { + // partitionEval[i++] = ExprNodeEvaluatorFactory.get(e); + //} + + tag = conf.getTag(); + tagByte[0] = (byte) tag; + tagWritable.set(tagByte[0]); + LOG.info("Using tag = " + tag); + + TableDesc keyTableDesc = conf.getKeySerializeInfo(); + keySerializer = (Serializer) keyTableDesc.getDeserializerClass() + .newInstance(); + keySerializer.initialize(null, keyTableDesc.getProperties()); + keyIsText = keySerializer.getSerializedClass().equals(Text.class); + + inputKeyDeserializer = (SerDe) ReflectionUtils.newInstance(keyTableDesc + .getDeserializerClass(), null); + inputKeyDeserializer.initialize(null, keyTableDesc.getProperties()); + outputKeyObjectInspector = inputKeyDeserializer.getObjectInspector(); + + + TableDesc valueTableDesc = conf.getValueSerializeInfo(); + valueSerializer = (Serializer) valueTableDesc.getDeserializerClass() + .newInstance(); + valueSerializer.initialize(null, valueTableDesc.getProperties()); + + + inputValueDeserializer = (SerDe) ReflectionUtils.newInstance( + valueTableDesc.getDeserializerClass(), null); + inputValueDeserializer.initialize(null, valueTableDesc + .getProperties()); + outputValueObjectInspector = inputValueDeserializer.getObjectInspector(); + + + + //firstRow = true; + + ObjectInspector rowInspector = inputObjInspectors[0]; + + keyObjectInspector = initEvaluatorsAndReturnStruct(keyEval, + distinctColIndices, + conf.getOutputKeyColumnNames(), numDistributionKeys, rowInspector); + valueObjectInspector = initEvaluatorsAndReturnStruct(valueEval, conf + .getOutputValueColumnNames(), rowInspector); + //partitionObjectInspectors = initEvaluators(partitionEval, rowInspector); + int numKeys = numDistinctExprs > 0 ? numDistinctExprs : 1; + int keyLen = numDistinctExprs > 0 ? numDistributionKeys + 1 : + numDistributionKeys; + cachedKeys = new Object[numKeys][keyLen]; + cachedValues = new Object[valueEval.length]; + assert cachedKeys.length == 1; + + ArrayList ois = new ArrayList(); + ois.add(outputKeyObjectInspector); + ois.add(outputValueObjectInspector); + ois.add(PrimitiveObjectInspectorFactory.writableByteObjectInspector); + + outputObjInspector = ObjectInspectorFactory + .getStandardStructObjectInspector(Arrays.asList(fieldNames), ois); + + System.out.println("in fake rs: numKeys: " + numKeys + ", keyLen:" + keyLen + ", numDistributionKeys:" + numDistributionKeys + + ", numDistinctExprs:" + numDistinctExprs); + System.out.println("childOperatorsTag.length:" + childOperatorsTag.length + " childOperatorsTag[0]:" + childOperatorsTag[0] + " tag:" + tag); + //childOperatorsTag[0] = tag; + + //firstRow = true; + + //peers = YSmartOptimizerUtils.findPeerFakeReduceSinkOperators(this); + + //LOG.info("# of peers fake reduce sink: " + peers.size()); + /* + for (i = 0; i < childOperatorsArray.length; i++) { + isGroupStarted.put(childOperatorsArray[i], new Boolean(false)); + } + */ + + LOG.info("Fake ReduceSink inputObjInspectors" + + ((StructObjectInspector) inputObjInspectors[0]).getTypeName()); //Yin + + LOG.info("Fake ReduceSink outputObjInspectors " + + this.getChildOperators().get(0).getParentOperators().indexOf(this) + + " " + ((StructObjectInspector) outputObjInspector).getTypeName()); //Yin + + initializeChildren(hconf); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + transient InspectableObject tempInspectableObject = new InspectableObject(); + transient HiveKey keyWritable = new HiveKey(); + transient Writable value; + + transient StructObjectInspector keyObjectInspector; + transient StructObjectInspector valueObjectInspector; + transient ObjectInspector[] partitionObjectInspectors; + + transient ObjectInspector outputKeyObjectInspector; + transient ObjectInspector outputValueObjectInspector; + transient ObjectInspector[] outputPartitionObjectInspectors; + + + //transient BytesWritable groupKey; + transient Object[][] cachedKeys; + transient Object[] cachedValues; + transient List> distinctColIndices; + + //boolean firstRow; + + transient Random random; + + private ArrayList forwardedRow = new ArrayList(3); + private Object keyObject; + private Object valueObject; + + + + /** + * Initializes array of ExprNodeEvaluator. Adds Union field for distinct + * column indices for group by. + * Puts the return values into a StructObjectInspector with output column + * names. + * + * If distinctColIndices is empty, the object inspector is same as + * {@link Operator#initEvaluatorsAndReturnStruct(ExprNodeEvaluator[], List, ObjectInspector)} + */ + protected static StructObjectInspector initEvaluatorsAndReturnStruct( + ExprNodeEvaluator[] evals, List> distinctColIndices, + List outputColNames, + int length, ObjectInspector rowInspector) + throws HiveException { + int inspectorLen = evals.length > length ? length + 1 : evals.length; + List sois = new ArrayList(inspectorLen); + + // keys + ObjectInspector[] fieldObjectInspectors = initEvaluators(evals, 0, length, rowInspector); + sois.addAll(Arrays.asList(fieldObjectInspectors)); + + if (evals.length > length) { + // union keys + List uois = new ArrayList(); + for (List distinctCols : distinctColIndices) { + List names = new ArrayList(); + List eois = new ArrayList(); + int numExprs = 0; + for (int i : distinctCols) { + names.add(HiveConf.getColumnInternalName(numExprs)); + eois.add(evals[i].initialize(rowInspector)); + numExprs++; + } + uois.add(ObjectInspectorFactory.getStandardStructObjectInspector(names, eois)); + } + UnionObjectInspector uoi = + ObjectInspectorFactory.getStandardUnionObjectInspector(uois); + sois.add(uoi); + } + return ObjectInspectorFactory.getStandardStructObjectInspector(outputColNames, sois ); + } + /* + HashMap, Boolean> isGroupStarted = + new HashMap, Boolean>(); + + public HashMap, Boolean> getIsGroupStarted(){ + return isGroupStarted; + } + */ + + private BytesWritable groupKey; + + @Override + public void processOp(Object row, int tag) throws HiveException { + + try { + // Evaluate the value + for (int i = 0; i < valueEval.length; i++) { + cachedValues[i] = valueEval[i].evaluate(row); + } + // Serialize the value + + value = valueSerializer.serialize(cachedValues, valueObjectInspector); + + valueObject = inputValueDeserializer.deserialize(value); + + // Evaluate the keys + Object[] distributionKeys = new Object[numDistributionKeys]; + for (int i = 0; i < numDistributionKeys; i++) { + distributionKeys[i] = keyEval[i].evaluate(row); + } + + if (numDistinctExprs > 0) { + // with distinct key(s) + for (int i = 0; i < numDistinctExprs; i++) { + System.arraycopy(distributionKeys, 0, cachedKeys[i], 0, numDistributionKeys); + Object[] distinctParameters = + new Object[distinctColIndices.get(i).size()]; + for (int j = 0; j < distinctParameters.length; j++) { + distinctParameters[j] = + keyEval[distinctColIndices.get(i).get(j)].evaluate(row); + } + cachedKeys[i][numDistributionKeys] = + new StandardUnion((byte)i, distinctParameters); + } + } else { + // no distinct key + System.arraycopy(distributionKeys, 0, cachedKeys[0], 0, numDistributionKeys); + } + + + for (int i = 0; i < cachedKeys.length; i++) { + + if (keyIsText) { + Text key = (Text) keySerializer.serialize(cachedKeys[i], + keyObjectInspector); + keyWritable.set(key.getBytes(), 0, key.getLength()); + + } else { + // Must be BytesWritable + BytesWritable key = (BytesWritable) keySerializer.serialize( + cachedKeys[i], keyObjectInspector); + keyWritable.set(key.getBytes(), 0, key.getLength()); + + } + + + + + + if (!keyWritable.equals(groupKey)) { + + try { + keyObject = inputKeyDeserializer.deserialize(keyWritable); + } catch (Exception e) { + throw new HiveException( + "Hive Runtime Error: Unable to deserialize reduce input key from " + + Utilities.formatBinaryString(keyWritable.get(), 0, + keyWritable.getSize()) + " with properties " + + keyTableDesc.getProperties(), e); + } + + + // If a operator wants to do some work at the beginning of a group + if (groupKey == null) { // the first group + groupKey = new BytesWritable(); + } else { + // If a operator wants to do some work at the end of a group + //l4j.trace("End Group"); + //LOG.info("YFRS end group on child " + childOperatorsArray[0].getName() + " " + childOperatorsArray[0].getIdentifier()); + //LOG.info("in YFRS End Group"); + // if the new group is already started, we should not end group + if(!keyWritable.equals(childOperatorsArray[0].getBytesWritableGroupKey())){ + //System.out.println("YFRS: endGroup()"); + childOperatorsArray[0].endGroup(); + } + + } + groupKey.set(keyWritable.get(), 0, keyWritable.getSize()); + + //childOperatorsArray[0].startGroup(); + //childOperatorsArray[0].setGroupKeyObject(keyObject); + + + if(!groupKey.equals(childOperatorsArray[0].getBytesWritableGroupKey())){ + //LOG.info("YFRS start group on child " + childOperatorsArray[0].getName() + " " + childOperatorsArray[0].getIdentifier()); + //LOG.info("in YFRS start Group"); + //System.out.println("YFRS: startGroup()"); + childOperatorsArray[0].startGroup(); + childOperatorsArray[0].setGroupKeyObject(keyObject); + childOperatorsArray[0].setBytesWritableGroupKey(groupKey); + } + + } + + + /* + forwardedRow.clear(); + forwardedRow.add(cachedKeys[i]); + forwardedRow.add(valueObject); + forwardedRow.add(tagWritable); + */ + + + + forwardedRow.clear(); + forwardedRow.add(keyObject); + forwardedRow.add(valueObject); + forwardedRow.add(tagWritable); + forward(forwardedRow, outputObjInspector); + } + } catch (SerDeException e1) { + // TODO Auto-generated catch block + e1.printStackTrace(); + } + + } + + /* + @Override + // comment this function when using processOp_old + public void setGroupKeyObject(Object keyObject) { + this.groupKeyObject = keyObject; + for (Operator op : childOperators) { + op.setGroupKeyObject(keyObject); + } + } + */ + + @Override + public void closeOp(boolean abort) throws HiveException { + if(!abort){ + //System.out.println("YFRS: getNumOfParentEndOP()" + childOperatorsArray[0].getNumOfParentEndOP()); + if(childOperatorsArray[0].getNumOfParentEndOP() == childOperatorsArray[0].getParentOperators().size() - 1){ + childOperatorsArray[0].endGroup(); + } + } + } + + + + @Override + public void startGroup() throws HiveException { + + + + } + + @Override + public void endGroup() throws HiveException { + + + } + + @Override + public void setGroupKeyObject(Object keyObject) { + + } + + + + public void processOp_old(Object row, int tag) throws HiveException { + + try { + // Evaluate the value + for (int i = 0; i < valueEval.length; i++) { + cachedValues[i] = valueEval[i].evaluate(row); + } + // Serialize the value + value = valueSerializer.serialize(cachedValues, valueObjectInspector); + valueObject = inputValueDeserializer.deserialize(value); + + // Evaluate the keys + Object[] distributionKeys = new Object[numDistributionKeys]; + for (int i = 0; i < numDistributionKeys; i++) { + distributionKeys[i] = keyEval[i].evaluate(row); + } + + if (numDistinctExprs > 0) { + // with distinct key(s) + for (int i = 0; i < numDistinctExprs; i++) { + System.arraycopy(distributionKeys, 0, cachedKeys[i], 0, numDistributionKeys); + Object[] distinctParameters = + new Object[distinctColIndices.get(i).size()]; + for (int j = 0; j < distinctParameters.length; j++) { + distinctParameters[j] = + keyEval[distinctColIndices.get(i).get(j)].evaluate(row); + } + cachedKeys[i][numDistributionKeys] = + new StandardUnion((byte)i, distinctParameters); + } + } else { + // no distinct key + System.arraycopy(distributionKeys, 0, cachedKeys[0], 0, numDistributionKeys); + } + + + for (int i = 0; i < cachedKeys.length; i++) { + + if (keyIsText) { + Text key = (Text) keySerializer.serialize(cachedKeys[i], + keyObjectInspector); + keyWritable.set(key.getBytes(), 0, key.getLength()); + + } else { + // Must be BytesWritable + BytesWritable key = (BytesWritable) keySerializer.serialize( + cachedKeys[i], keyObjectInspector); + keyWritable.set(key.getBytes(), 0, key.getLength()); + + } + + //keyObject = inputKeyDeserializer.deserialize(keyWritable); + + + //keyObject = cachedKeys[i]; + //valueObject = cachedValues; + + //groupKey = childOperatorsArray[0].getGroupKeyObject(); + + /* + if (!keyWritable.equals(groupKey)) { + // If a operator wants to do some work at the beginning of a group + if (groupKey == null) { // the first group + groupKey = new BytesWritable(); + } else { + // If a operator wants to do some work at the end of a group + LOG.info("in YFRS End Group"); + childOperatorsArray[0].endGroup(); + } + + try { + keyObject = inputKeyDeserializer.deserialize(keyWritable); + } catch (Exception e) { + throw new HiveException( + "Hive Runtime Error: Unable to deserialize reduce input key from " + + Utilities.formatBinaryString(keyWritable.get(), 0, + keyWritable.getSize()) + " with properties " + + keyTableDesc.getProperties(), e); + } + groupKey.set(keyWritable.get(), 0, keyWritable.getSize()); + + if(!keyObject.equals(childOperatorsArray[0].getGroupKeyObject())){ + LOG.info("in YFRS start Group"); + childOperatorsArray[0].startGroup(); + childOperatorsArray[0].setGroupKeyObject(keyObject); + } + } + */ + + + forwardedRow.clear(); + forwardedRow.add(keyObject); + forwardedRow.add(valueObject); + forwardedRow.add(tagWritable); + forward(forwardedRow, outputObjInspector); + } + } catch (SerDeException e) { + throw new HiveException(e); + } + + } + + + + + + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return new String("YFReduceSink"); + } + + @Override + public OperatorType getType() { + return OperatorType.YSMARTFAKEREDUCESINK; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartReduceSinkOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartReduceSinkOperator.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartReduceSinkOperator.java (revision 0) @@ -0,0 +1,351 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.io.HiveKey; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.plan.YSmartReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.Serializer; +import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspector.StandardUnion; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; + +/** + * Reduce Sink Operator sends output to the reduce stage. + **/ +public class YSmartReduceSinkOperator extends TerminalOperator + implements Serializable { + + private static final long serialVersionUID = 1L; + + /** + * The evaluators for the key columns. Key columns decide the sort order on + * the reducer side. Key columns are passed to the reducer in the "key". + */ + protected transient ExprNodeEvaluator[] keyEval; + /** + * The evaluators for the value columns. Value columns are passed to reducer + * in the "value". + */ + protected transient ExprNodeEvaluator[] valueEval; + /** + * The evaluators for the partition columns (CLUSTER BY or DISTRIBUTE BY in + * Hive language). Partition columns decide the reducer that the current row + * goes to. Partition columns are not passed to reducer. + */ + protected transient ExprNodeEvaluator[] partitionEval; + + // TODO: we use MetadataTypedColumnsetSerDe for now, till DynamicSerDe is + // ready + transient Serializer keySerializer; + transient boolean keyIsText; + transient Serializer valueSerializer; + transient int tag; + transient byte[] tagByte = new byte[1]; + transient protected int numDistributionKeys; + transient protected int numDistinctExprs; + private final ArrayList opTags = new ArrayList(); //Yin: Tags used for indicating which ops this row does not belong to + private final byte[] opTagsByte = new byte[1]; + // layout of opTagsInt from left(most significant bit) to right(least significant bit), op7, op6, op5, op4, op3, op2, op1, op0 + // Thus, for example, if there are four ops(op0-op3), and notOpTags contains 1 and 2, the opTagsInt is 00001001 + + + public void setOpTags(ArrayList opTags){ + this.opTags.addAll(opTags); + int opTagsInt = 0; + int tmp = 1; + for (Integer opTag: opTags){ + opTagsInt += tmp << opTag.intValue(); + } + opTagsByte[0] = (byte) opTagsInt; + } + + public ArrayList getOpTags(){ + return this.opTags; + } + + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + + try { + keyEval = new ExprNodeEvaluator[conf.getKeyCols().size()]; + int i = 0; + for (ExprNodeDesc e : conf.getKeyCols()) { + keyEval[i++] = ExprNodeEvaluatorFactory.get(e); + } + + numDistributionKeys = conf.getNumDistributionKeys(); + distinctColIndices = conf.getDistinctColumnIndices(); + numDistinctExprs = distinctColIndices.size(); + + valueEval = new ExprNodeEvaluator[conf.getValueCols().size()]; + i = 0; + for (ExprNodeDesc e : conf.getValueCols()) { + valueEval[i++] = ExprNodeEvaluatorFactory.get(e); + } + + partitionEval = new ExprNodeEvaluator[conf.getPartitionCols().size()]; + i = 0; + for (ExprNodeDesc e : conf.getPartitionCols()) { + partitionEval[i++] = ExprNodeEvaluatorFactory.get(e); + } + + tag = conf.getTag(); + tagByte[0] = (byte) tag; + LOG.info("Using tag = " + tag); + + TableDesc keyTableDesc = conf.getKeySerializeInfo(); + keySerializer = (Serializer) keyTableDesc.getDeserializerClass() + .newInstance(); + keySerializer.initialize(null, keyTableDesc.getProperties()); + keyIsText = keySerializer.getSerializedClass().equals(Text.class); + + TableDesc valueTableDesc = conf.getValueSerializeInfo(); + valueSerializer = (Serializer) valueTableDesc.getDeserializerClass() + .newInstance(); + valueSerializer.initialize(null, valueTableDesc.getProperties()); + + firstRow = true; + + if (conf.getStaticOpTags() != null){ + setOpTags(conf.getStaticOpTags()); + } + + initializeChildren(hconf); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + } + + transient InspectableObject tempInspectableObject = new InspectableObject(); + transient HiveKey keyWritable = new HiveKey(); + transient Writable value; + + transient StructObjectInspector keyObjectInspector; + transient StructObjectInspector valueObjectInspector; + transient ObjectInspector[] partitionObjectInspectors; + + transient Object[][] cachedKeys; + transient Object[] cachedValues; + transient List> distinctColIndices; + + boolean firstRow; + + transient Random random; + + /** + * Initializes array of ExprNodeEvaluator. Adds Union field for distinct + * column indices for group by. + * Puts the return values into a StructObjectInspector with output column + * names. + * + * If distinctColIndices is empty, the object inspector is same as + * {@link Operator#initEvaluatorsAndReturnStruct(ExprNodeEvaluator[], List, ObjectInspector)} + */ + protected static StructObjectInspector initEvaluatorsAndReturnStruct( + ExprNodeEvaluator[] evals, List> distinctColIndices, + List outputColNames, + int length, ObjectInspector rowInspector) + throws HiveException { + int inspectorLen = evals.length > length ? length + 1 : evals.length; + List sois = new ArrayList(inspectorLen); + + // keys + ObjectInspector[] fieldObjectInspectors = initEvaluators(evals, 0, length, rowInspector); + sois.addAll(Arrays.asList(fieldObjectInspectors)); + + if (evals.length > length) { + // union keys + List uois = new ArrayList(); + for (List distinctCols : distinctColIndices) { + List names = new ArrayList(); + List eois = new ArrayList(); + int numExprs = 0; + for (int i : distinctCols) { + names.add(HiveConf.getColumnInternalName(numExprs)); + eois.add(evals[i].initialize(rowInspector)); + numExprs++; + } + uois.add(ObjectInspectorFactory.getStandardStructObjectInspector(names, eois)); + } + UnionObjectInspector uoi = + ObjectInspectorFactory.getStandardUnionObjectInspector(uois); + sois.add(uoi); + } + return ObjectInspectorFactory.getStandardStructObjectInspector(outputColNames, sois ); + } + + @Override + public void processOp(Object row, int tag) throws HiveException { + try { + ObjectInspector rowInspector = inputObjInspectors[tag]; + if (firstRow) { + firstRow = false; + keyObjectInspector = initEvaluatorsAndReturnStruct(keyEval, + distinctColIndices, + conf.getOutputKeyColumnNames(), numDistributionKeys, rowInspector); + valueObjectInspector = initEvaluatorsAndReturnStruct(valueEval, conf + .getOutputValueColumnNames(), rowInspector); + partitionObjectInspectors = initEvaluators(partitionEval, rowInspector); + int numKeys = numDistinctExprs > 0 ? numDistinctExprs : 1; + int keyLen = numDistinctExprs > 0 ? numDistributionKeys + 1 : + numDistributionKeys; + cachedKeys = new Object[numKeys][keyLen]; + cachedValues = new Object[valueEval.length]; + } + + // Evaluate the HashCode + int keyHashCode = 0; + if (partitionEval.length == 0) { + // If no partition cols, just distribute the data uniformly to provide + // better + // load balance. If the requirement is to have a single reducer, we + // should set + // the number of reducers to 1. + // Use a constant seed to make the code deterministic. + if (random == null) { + random = new Random(12345); + } + keyHashCode = random.nextInt(); + } else { + for (int i = 0; i < partitionEval.length; i++) { + Object o = partitionEval[i].evaluate(row); + keyHashCode = keyHashCode * 31 + + ObjectInspectorUtils.hashCode(o, partitionObjectInspectors[i]); + } + } + + // Evaluate the value + for (int i = 0; i < valueEval.length; i++) { + cachedValues[i] = valueEval[i].evaluate(row); + } + // Serialize the value + value = valueSerializer.serialize(cachedValues, valueObjectInspector); + + // Evaluate the keys + Object[] distributionKeys = new Object[numDistributionKeys]; + for (int i = 0; i < numDistributionKeys; i++) { + distributionKeys[i] = keyEval[i].evaluate(row); + } + + if (numDistinctExprs > 0) { + // with distinct key(s) + for (int i = 0; i < numDistinctExprs; i++) { + System.arraycopy(distributionKeys, 0, cachedKeys[i], 0, numDistributionKeys); + Object[] distinctParameters = + new Object[distinctColIndices.get(i).size()]; + for (int j = 0; j < distinctParameters.length; j++) { + distinctParameters[j] = + keyEval[distinctColIndices.get(i).get(j)].evaluate(row); + } + cachedKeys[i][numDistributionKeys] = + new StandardUnion((byte)i, distinctParameters); + } + } else { + // no distinct key + System.arraycopy(distributionKeys, 0, cachedKeys[0], 0, numDistributionKeys); + } + // Serialize the keys and append the tag + //System.out.println("in YRS: opTag" + opTagsByte[0] + " ,tag:" + tag + ", tagByte:" + tagByte[0]); + for (int i = 0; i < cachedKeys.length; i++) { + if (keyIsText) { + Text key = (Text) keySerializer.serialize(cachedKeys[i], + keyObjectInspector); + if (tag == -1) { + keyWritable.set(key.getBytes(), 0, key.getLength()); + } else { + int keyLength = key.getLength(); + //keyWritable.setSize(keyLength + 1); + keyWritable.setSize(keyLength + 2); + System.arraycopy(key.getBytes(), 0, keyWritable.get(), 0, keyLength); + keyWritable.get()[keyLength] = opTagsByte[0]; + //System.out.println("in YRS: setting opTags " + keyWritable.get()[keyLength]);//Yin + keyWritable.get()[keyLength + 1] = tagByte[0]; + } + } else { + // Must be BytesWritable + BytesWritable key = (BytesWritable) keySerializer.serialize( + cachedKeys[i], keyObjectInspector); + if (tag == -1) { + keyWritable.set(key.getBytes(), 0, key.getLength()); + } else { + int keyLength = key.getLength(); + //keyWritable.setSize(keyLength + 1); + keyWritable.setSize(keyLength + 2); + System.arraycopy(key.getBytes(), 0, keyWritable.get(), 0, keyLength); + keyWritable.get()[keyLength] = opTagsByte[0]; + //System.out.println("in YRS: setting opTags " + keyWritable.get()[keyLength]);//Yin + keyWritable.get()[keyLength + 1] = tagByte[0]; + } + } + keyWritable.setHashCode(keyHashCode); + if (out != null) { + out.collect(keyWritable, value); + // Since this is a terminal operator, update counters explicitly - + // forward is not called + if (counterNameToEnum != null) { + ++outputRows; + if (outputRows % 1000 == 0) { + incrCounter(numOutputRowsCntr, outputRows); + outputRows = 0; + } + } + } + } + } catch (SerDeException e) { + throw new HiveException(e); + } catch (IOException e) { + throw new HiveException(e); + } + } + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return new String("YRS"); + } + + @Override + public OperatorType getType() { + return OperatorType.YSMARTREDUCESINK; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartDispatchOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartDispatchOperator.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartDispatchOperator.java (revision 0) @@ -0,0 +1,393 @@ +package org.apache.hadoop.hive.ql.exec; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map.Entry; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.SelectDesc; +import org.apache.hadoop.hive.ql.plan.YSmartDispatchDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.io.BytesWritable; + +public class YSmartDispatchOperator extends Operator implements Serializable{ + + + + /** + * + */ + private static final long serialVersionUID = 1L; + + + + + private static String[] fieldNames; + static { + ArrayList fieldNameArray = new ArrayList(); + for (Utilities.ReduceField r : Utilities.ReduceField.values()) { + fieldNameArray.add(r.toString()); + } + fieldNames = fieldNameArray.toArray(new String[0]); + } + + protected static class DispatchHandler{ + + protected Log l4j = LogFactory.getLog(this.getClass().getName()); + + private ObjectInspector[] inputObjInspector; + private ObjectInspector outputObjInspector; + private ObjectInspector keyObjInspector; + private ObjectInspector valueObjInspector; + private byte inputTag; + private byte outputTag; + private ByteWritable outputTagByteWritable; + private SelectDesc selectDesc; + private SelectDesc keySelectDesc; + private ExprNodeEvaluator[] keyEval; + private ExprNodeEvaluator[] eval; + //private Object[] keyOutput; + //private Object[] valueOutput; + //private ArrayList outputRow; + + public DispatchHandler(ObjectInspector[] inputObjInspector, byte inputTag, byte outputTag, SelectDesc selectDesc, SelectDesc keySelectDesc) throws HiveException{ + this.inputObjInspector = inputObjInspector; + //System.out.println("inputObjInspector:" + inputObjInspector.toString()); + assert this.inputObjInspector.length == 1; + this.inputTag = inputTag; + this.outputTag = outputTag; + this.selectDesc = selectDesc; + this.keySelectDesc = keySelectDesc; + //this.outputRow = new ArrayList(3); + this.outputTagByteWritable = new ByteWritable(outputTag); + init(); + } + + private void init() throws HiveException{ + ArrayList ois = new ArrayList(); + if (keySelectDesc.isSelStarNoCompute()){ + ois.add((ObjectInspector) ((ArrayList)inputObjInspector[0]).get(0)); + }else{ + ArrayList colList = this.keySelectDesc.getColList(); + keyEval = new ExprNodeEvaluator[colList.size()]; + for (int k = 0; k < colList.size(); k++) { + assert (colList.get(k) != null); + keyEval[k] = ExprNodeEvaluatorFactory.get(colList.get(k)); + } + //keyOutput = new Object[keyEval.length]; + keyObjInspector = initEvaluatorsAndReturnStruct(keyEval, keySelectDesc + .getOutputColumnNames(), ((StandardStructObjectInspector) inputObjInspector[0]).getAllStructFieldRefs().get(0).getFieldObjectInspector()); + + ois.add(keyObjInspector); + l4j.info("Key: input tag " + (int)inputTag + ", output tag " + (int)outputTag + ", SELECT inputOIForThisTag" + + ((StructObjectInspector) inputObjInspector[0]).getTypeName()); //Yin + + } + if (selectDesc.isSelStarNoCompute()){ + ois.add((ObjectInspector) ((ArrayList)inputObjInspector[0]).get(1)); + }else{ + ArrayList colList = this.selectDesc.getColList(); + eval = new ExprNodeEvaluator[colList.size()]; + for (int k = 0; k < colList.size(); k++) { + assert (colList.get(k) != null); + eval[k] = ExprNodeEvaluatorFactory.get(colList.get(k)); + } + //valueOutput = new Object[eval.length]; + valueObjInspector = initEvaluatorsAndReturnStruct(eval, selectDesc + .getOutputColumnNames(), ((StandardStructObjectInspector) inputObjInspector[0]).getAllStructFieldRefs().get(1).getFieldObjectInspector()); + + ois.add(valueObjInspector); + l4j.info("input tag " + (int)inputTag + ", output tag " + (int)outputTag + ", SELECT inputOIForThisTag" + + ((StructObjectInspector) inputObjInspector[0]).getTypeName()); //Yin + + } + ois.add(PrimitiveObjectInspectorFactory.writableByteObjectInspector); + outputObjInspector = ObjectInspectorFactory + .getStandardStructObjectInspector(Arrays.asList(fieldNames), ois); + l4j.info("input tag " + (int)inputTag + ", output tag " + (int)outputTag + ", SELECT outputObjInspector" + + ((StructObjectInspector) outputObjInspector).getTypeName()); //Yin + + } + + + + public ObjectInspector getOutputObjInspector(){ + return outputObjInspector; + } + + public Object process(Object row) throws HiveException{ + Object[] keyOutput = new Object[keyEval.length]; + Object[] valueOutput = new Object[eval.length]; + //outputRow.clear(); + ArrayList outputRow = new ArrayList(3); + List thisRow = (List)row; + + if(keySelectDesc.isSelStarNoCompute()){ + outputRow.add(thisRow.get(0)); + }else{ + Object key = thisRow.get(0); + for (int j = 0; j < keyEval.length; j++) { + try { + keyOutput[j] = keyEval[j].evaluate(key); + //Yin + //System.out.println("output[i].toString:" + output[i].toString() + "|row.toString" + row.toString()); + } catch (HiveException e) { + throw e; + } catch (RuntimeException e) { + throw new HiveException("Error evaluating " + + keySelectDesc.getColList().get(j).getExprString(), e); + } + } + outputRow.add(keyOutput); + } + + if (selectDesc.isSelStarNoCompute()){ + //return thisRow.get(0); + //return row; + //TODO: uncomment these three lines after test + + outputRow.add(thisRow.get(1)); + + }else{ + Object value = thisRow.get(1); + for (int j = 0; j < eval.length; j++) { + try { + valueOutput[j] = eval[j].evaluate(value); + //Yin + //System.out.println("output[i].toString:" + output[i].toString() + "|row.toString" + row.toString()); + } catch (HiveException e) { + throw e; + } catch (RuntimeException e) { + throw new HiveException("Error evaluating " + + selectDesc.getColList().get(j).getExprString(), e); + } + } + + outputRow.add(valueOutput); + + } + outputRow.add(outputTagByteWritable); + return outputRow; + } + + } + + private HashMap> dispatchConf; //inputTag->(Child->outputTag) + private HashMap> dispatchValueSelectDescConf; //inputTag->(Child->SelectDesc) + private HashMap> dispatchKeySelectDescConf; //inputTag->(Child->SelectDesc) + private HashMap> dispatchHandlers; //inputTag->(Child->DispatchHandler) + private HashMap> child2OutputTag2DispatchHandlers; //Child->(outputTag->DispatchHandler) + private HashMap childInputObjInspectors; //Child->Child's inputObjInspectors + @Override + protected void initializeOp(Configuration hconf) throws HiveException { + + dispatchConf = conf.getDispatchConf(); + System.out.println("dispatchConf.toString(): " + dispatchConf.toString()); + dispatchValueSelectDescConf = conf.getDispatchValueSelectDescConf(); + dispatchKeySelectDescConf = conf.getDispatchKeySelectDescConf(); + dispatchHandlers = new HashMap>(); + for(Entry> entry: dispatchConf.entrySet()){ + HashMap tmp = new HashMap(); + for(Entry child2outputTag: entry.getValue().entrySet()){ + tmp.put(child2outputTag.getKey(), + new DispatchHandler(new ObjectInspector[]{inputObjInspectors[entry.getKey()]}, + entry.getKey().byteValue(), child2outputTag.getValue().byteValue(), + dispatchValueSelectDescConf.get(entry.getKey()).get(child2outputTag.getKey()), + dispatchKeySelectDescConf.get(entry.getKey()).get(child2outputTag.getKey()))); + } + dispatchHandlers.put(entry.getKey(), tmp); + } + + child2OutputTag2DispatchHandlers = new HashMap>(); + for(Entry> entry: dispatchConf.entrySet()){ + for(Entry child2outputTag: entry.getValue().entrySet()){ + if (!child2OutputTag2DispatchHandlers.containsKey(child2outputTag.getKey())){ + child2OutputTag2DispatchHandlers.put(child2outputTag.getKey(), new HashMap()); + } + child2OutputTag2DispatchHandlers.get(child2outputTag.getKey()). + put(child2outputTag.getValue(), dispatchHandlers.get(entry.getKey()).get(child2outputTag.getKey())); + } + } + + System.out.println("child2OutputTag2DispatchHandlers " + child2OutputTag2DispatchHandlers.toString()); + childInputObjInspectors = new HashMap(); + for(Entry> entry: child2OutputTag2DispatchHandlers.entrySet()){ + Integer l = Collections.max(entry.getValue().keySet()); + ObjectInspector[] childObjInspectors = new ObjectInspector[l.intValue() + 1]; + for(Entry e: entry.getValue().entrySet()){ + if(e.getKey().intValue() == -1){ + assert childObjInspectors.length == 1; + childObjInspectors[0] = e.getValue().getOutputObjInspector(); + //}else if (childObjInspectors.length == 1){ + // childObjInspectors[0] = e.getValue().getOutputObjInspector(); + }else{ + childObjInspectors[e.getKey().intValue()] = e.getValue().getOutputObjInspector(); + } + } + //System.out.println(entry.getKey() + " " + childObjInspectors.length);//TEST THIS + + + childInputObjInspectors.put(entry.getKey(), childObjInspectors); + } + + initializeChildren(hconf); + } + +//Each child should has its own outputObjInspector + @Override + protected void initializeChildren(Configuration hconf) throws HiveException { + state = State.INIT; + LOG.info("Operator " + id + " " + getName() + " initialized"); + if (childOperators == null) { + return; + } + LOG.info("Initializing children of " + id + " " + getName()); + for (int i = 0; i < childOperatorsArray.length; i++) { + LOG.info("Initializing child " + i + " " + childOperatorsArray[i].getIdentifier() + " " + + childOperatorsArray[i].getName() + + " " + childInputObjInspectors.get(i).length); + childOperatorsArray[i].initialize(hconf, childInputObjInspectors.get(i)); + if (reporter != null) { + childOperatorsArray[i].setReporter(reporter); + } + } + } + + + private int opTags; + private int inputTag; + + @Override + public void processOp(Object row, int tag) throws HiveException { + ArrayList thisRow = (ArrayList)row; + //System.out.println(thisRow.toString()); + assert thisRow.size() == 4; + opTags = ((ByteWritable)thisRow.get(3)).get(); + + inputTag = (int)((ByteWritable)thisRow.get(2)).get(); + + /* + ArrayList forwardedRow = new ArrayList(3); + try { + + BytesWritable key = new BytesWritable(); + key.set(keyWritableForYSMartDispatch.get(), 0, keyWritableForYSMartDispatch.getLength()); + BytesWritable value = new BytesWritable(); + value.set(valueWritableForYSMartDispatch.get(), 0, valueWritableForYSMartDispatch.getLength()); + + forwardedRow.add(keyDeserializerForYSMartDispatch.deserialize(key)); + + forwardedRow.add(valueSerDeForYSMartDispatch.deserialize(value)); + forwardedRow.add(new ByteWritable((byte)inputTag)); + + } catch (SerDeException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + */ + + forward(thisRow.subList(0, 3), inputObjInspectors[inputTag]); + + } + + + + @Override + public void forward(Object row, ObjectInspector rowInspector) + throws HiveException { + + //System.out.println("in DISP... opTags:" + opTags + ", # of children:" + childOperatorsArray.length + // + ", dispatchConf:" + dispatchConf.toString()); + + if ((++outputRows % 1000) == 0) { + if (counterNameToEnum != null) { + incrCounter(numOutputRowsCntr, outputRows); + outputRows = 0; + } + } + + if (childOperatorsArray == null && childOperators != null) { + throw new HiveException( + "Internal Hive error during operator initialization."); + } + + if ((childOperatorsArray == null) || (getDone())) { + return; + } + + + int childrenDone = 0; + int forwardFLag = 1; + assert childOperatorsArray.length <= 8; + for (int i = 0; i < childOperatorsArray.length; i++) { + Operator o = childOperatorsArray[i]; + if (o.getDone()) { + childrenDone++; + } else { + if ((opTags & (forwardFLag << i)) != 0){ + o.process(dispatchHandlers.get(inputTag).get(i).process(row), dispatchConf.get(inputTag).get(i)); + + + + } + } + } + + // if all children are done, this operator is also done + if (childrenDone == childOperatorsArray.length) { + setDone(true); + } + } + + + + + @Override + public void setBytesWritableGroupKey(BytesWritable groupKey) { + if(bytesWeitableGroupKey == null){ + bytesWeitableGroupKey = new BytesWritable(); + } + bytesWeitableGroupKey.set(groupKey.get(), 0, groupKey.getSize()); + for(Operator op : childOperators){ + op.setBytesWritableGroupKey(bytesWeitableGroupKey); + } + } + + @Override + public void setGroupKeyObject(Object keyObject) { + this.groupKeyObject = keyObject; + for(Operator op : childOperators){ + op.setGroupKeyObject(keyObject); + } + } + + + + @Override + public OperatorType getType() { + return OperatorType.YSmartDispatch; + } + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return new String("YDP"); + } + +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartManualForwardOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartManualForwardOperator.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/YSmartManualForwardOperator.java (revision 0) @@ -0,0 +1,76 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.io.Serializable; + +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.YSmartManualForwardDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; + +/** + * YSmartManualForwardOperator. Collect row and forward it manually + **/ +public class YSmartManualForwardOperator extends Operator implements + Serializable { + private static final long serialVersionUID = 1L; + + private Object row; + private int tag; + + public Object getRow(){ + return this.row; + } + + public int getTag(){ + return this.tag; + } + + @Override + public void processOp(Object row, int tag) throws HiveException { + //System.out.println("in YMF: row: " + row.toString()); + this.row = row; + this.tag = tag; + } + + public void forwardIt() throws HiveException { + forward(row, inputObjInspectors[tag]); + } + + public void clean(){ + this.row = null; + this.tag = -1; + } + + /** + * @return the name of the operator + */ + @Override + public String getName() { + return new String("YMF"); + } + + @Override + public OperatorType getType() { + return OperatorType.YSmartManualForward; + } + + + +} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java (revision 1132812) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java (working copy) @@ -42,7 +42,14 @@ import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.hive.ql.plan.UDTFDesc; import org.apache.hadoop.hive.ql.plan.UnionDesc; +import org.apache.hadoop.hive.ql.plan.YSmartCompositeDesc; +import org.apache.hadoop.hive.ql.plan.YSmartDispatchDesc; +import org.apache.hadoop.hive.ql.plan.YSmartFakeReduceSinkDesc; +import org.apache.hadoop.hive.ql.plan.YSmartForwardDesc; +import org.apache.hadoop.hive.ql.plan.YSmartManualForwardDesc; +import org.apache.hadoop.hive.ql.plan.YSmartReduceSinkDesc; + /** * OperatorFactory. * @@ -91,6 +98,15 @@ HashTableDummyOperator.class)); opvec.add(new OpTuple(HashTableSinkDesc.class, HashTableSinkOperator.class)); + + opvec.add(new OpTuple(YSmartCompositeDesc.class, YSmartCompositeOperator.class)); + opvec.add(new OpTuple(YSmartManualForwardDesc.class, YSmartManualForwardOperator.class)); + opvec.add(new OpTuple(YSmartReduceSinkDesc.class, YSmartReduceSinkOperator.class)); + opvec.add(new OpTuple(YSmartDispatchDesc.class, YSmartDispatchOperator.class)); + opvec.add(new OpTuple(YSmartFakeReduceSinkDesc.class, YSmartFakeReduceSinkOperator.class)); + opvec.add(new OpTuple(YSmartForwardDesc.class, YSmartForwardOperator.class)); + + } public static Operator get(Class opClass) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java (revision 1132812) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java (working copy) @@ -38,6 +38,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.mapred.Counters; import org.apache.hadoop.mapred.OutputCollector; @@ -55,6 +56,7 @@ protected List> childOperators; protected List> parentOperators; + protected int numberOfParentOperators; //Yin protected String operatorId; /** * List of counter names associated with the operator. It contains the @@ -345,6 +347,12 @@ } } + if(parentOperators == null){ + numberOfParentOperators = 0; + }else{ + numberOfParentOperators = parentOperators.size(); + } + if (inputObjInspectors.length == 0) { throw new HiveException("Internal Error during operator initialization."); } @@ -526,50 +534,60 @@ return true; } + protected int numOfParentEndOP = 0; //Yin + + public int getNumOfParentEndOP(){ + return numOfParentEndOP; + } + // This close() function does not need to be synchronized // since it is called by its parents' main thread, so no // more than 1 thread should call this close() function. public void close(boolean abort) throws HiveException { - if (state == State.CLOSE) { - return; - } - - // check if all parents are finished - if (!allInitializedParentsAreClosed()) { - return; - } - - // set state as CLOSE as long as all parents are closed - // state == CLOSE doesn't mean all children are also in state CLOSE - state = State.CLOSE; - LOG.info(id + " finished. closing... "); - - if (counterNameToEnum != null) { - incrCounter(numInputRowsCntr, inputRows); - incrCounter(numOutputRowsCntr, outputRows); - incrCounter(timeTakenCntr, totalTime); - } - - LOG.info(id + " forwarded " + cntr + " rows"); - - // call the operator specific close routine - closeOp(abort); - - try { - logStats(); - if (childOperators == null) { + if(numberOfParentOperators ==0 || numOfParentEndOP == numberOfParentOperators-1){ + if (state == State.CLOSE) { return; } - - for (Operator op : childOperators) { - op.close(abort); + + // check if all parents are finished + if (!allInitializedParentsAreClosed()) { + return; } - - LOG.info(id + " Close done"); - } catch (HiveException e) { - e.printStackTrace(); - throw e; + + // set state as CLOSE as long as all parents are closed + // state == CLOSE doesn't mean all children are also in state CLOSE + state = State.CLOSE; + LOG.info(id + " finished. closing... "); + + if (counterNameToEnum != null) { + incrCounter(numInputRowsCntr, inputRows); + incrCounter(numOutputRowsCntr, outputRows); + incrCounter(timeTakenCntr, totalTime); + } + + LOG.info(id + " forwarded " + cntr + " rows"); + + // call the operator specific close routine + closeOp(abort); + + try { + logStats(); + if (childOperators == null) { + return; + } + + for (Operator op : childOperators) { + op.close(abort); + } + + LOG.info(id + " Close done"); + } catch (HiveException e) { + e.printStackTrace(); + throw e; + } + }else{ + numOfParentEndOP++; } } @@ -1314,4 +1332,17 @@ public void cleanUpInputFileChangedOp() throws HiveException { } + protected BytesWritable bytesWeitableGroupKey; //Yin + + public void setBytesWritableGroupKey(BytesWritable groupKey) { + if(bytesWeitableGroupKey == null){ + bytesWeitableGroupKey = new BytesWritable(); + } + bytesWeitableGroupKey.set(groupKey.get(), 0, groupKey.getSize()); + } + + public BytesWritable getBytesWritableGroupKey() { + return bytesWeitableGroupKey; + } + } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExecReducer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecReducer.java (revision 1132812) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecReducer.java (working copy) @@ -61,6 +61,7 @@ private Reporter rp; private boolean abort = false; private boolean isTagged = false; + private boolean isOpsTagged = false; private long cntr = 0; private long nextCntr = 1; @@ -116,6 +117,7 @@ reducer.setParentOperators(null); // clear out any parents as reducer is the // root isTagged = gWork.getNeedsTagging(); + isOpsTagged = gWork.getNeedsOpsTagging(); try { keyTableDesc = gWork.getKeyDesc(); inputKeyDeserializer = (SerDe) ReflectionUtils.newInstance(keyTableDesc @@ -164,8 +166,9 @@ private BytesWritable groupKey; - ArrayList row = new ArrayList(3); + ArrayList row = new ArrayList(4); ByteWritable tag = new ByteWritable(); + ByteWritable opTags = new ByteWritable(); //Yin public void reduce(Object key, Iterator values, OutputCollector output, Reporter reporter) throws IOException { @@ -187,6 +190,15 @@ tag.set(keyWritable.get()[size]); keyWritable.setSize(size); } + + opTags.set((byte)0); + if (isOpsTagged){ + //Yin: remove opTags + int size = keyWritable.getSize() - 1; + opTags.set(keyWritable.get()[size]); + keyWritable.setSize(size); + //System.out.println("in execReduce: opTags: " + opTags.get()); + } if (!keyWritable.equals(groupKey)) { // If a operator wants to do some work at the beginning of a group @@ -212,6 +224,7 @@ l4j.trace("Start Group"); reducer.startGroup(); reducer.setGroupKeyObject(keyObject); + reducer.setBytesWritableGroupKey(groupKey);//Yin } // System.err.print(keyObject.toString()); while (values.hasNext()) { @@ -234,6 +247,7 @@ row.add(valueObject[tag.get()]); // The tag is not used any more, we should remove it. row.add(tag); + row.add(opTags); if (isLogInfoEnabled) { cntr++; if (cntr == nextCntr) { Index: ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartManualForwardDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartManualForwardDesc.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartManualForwardDesc.java (revision 0) @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.io.Serializable; + +/** + * ForwardDesc. + * + */ +@Explain(displayName = "YSmart Manual Forward") +public class YSmartManualForwardDesc implements Serializable { + private static final long serialVersionUID = 1L; + + public YSmartManualForwardDesc() { + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartCompositeDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartCompositeDesc.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartCompositeDesc.java (revision 0) @@ -0,0 +1,107 @@ +package org.apache.hadoop.hive.ql.plan; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.YSmartManualForwardOperator; +import org.apache.hadoop.hive.ql.exec.YSmartReduceSinkOperator; + + +/** + * YSmart composite operator Descriptor implementation. + * + */ +@Explain(displayName = "YSmart Composite Operator") +public class YSmartCompositeDesc implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + private YSmartReduceSinkOperator correspondingYRS; + + public YSmartCompositeDesc(){ + + } + + public YSmartCompositeDesc(YSmartReduceSinkOperator correspondingYRS){ + this.correspondingYRS = correspondingYRS; + } + + public void setCorrespondingYRS(YSmartReduceSinkOperator correspondingYRS){ + this.correspondingYRS = correspondingYRS; + } + + public YSmartReduceSinkOperator getCorrespondingYRS(){ + return correspondingYRS; + } + + private int[] allOpTags; + private ArrayList> topInternalOps = new ArrayList>(); + private ArrayList ymfOps = new ArrayList();; + + public void setYmfOps(ArrayList ymfOps){ + this.ymfOps = ymfOps; + } + + public ArrayList getYmfOps(){ + return ymfOps; + } + + public void setTopInternalOps(ArrayList> topInternalOps){ + this.topInternalOps = topInternalOps; + } + + public ArrayList> getTopInternalOps(){ + return topInternalOps; + } + + public void setAllOpTags(int[] allOpTags){ + this.allOpTags = allOpTags; + } + + public int[] getAllOpTags(){ + return allOpTags; + } + + + public void setInternalNodes(ArrayList> topInternalOps, + ArrayList> bottomInternalOps, int[] allOpTags){ + this.allOpTags = allOpTags; + // the size of topInternalOps and bottomInternalOps should be same. + this.topInternalOps.addAll(topInternalOps); + //List> childrenOfThisOp = new ArrayList>(); + for (Operator op: bottomInternalOps){ + //List> originalChilren = op.getChildOperators(); + YSmartManualForwardOperator ymf = new YSmartManualForwardOperator(); + ymf.setChildOperators(null); + ymfOps.add(ymf); + List> newChildren = + new ArrayList>(); + newChildren.add(ymf); + op.setChildOperators(newChildren); + ymf.setParentOperators(Utilities.makeList(op)); + /* + for (Operator ch : originalChilren) { + System.out.println("ch:" + ch.toString()); + List> parentOperators = ch + .getParentOperators(); + int pos = parentOperators.indexOf(op); + assert pos != -1; + parentOperators.remove(pos); + parentOperators.add(pos, this); // add this op as the parent + } + childrenOfThisOp.addAll(originalChilren); + */ + } + + + } + + + +} Index: ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartForwardDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartForwardDesc.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartForwardDesc.java (revision 0) @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.io.Serializable; + +/** + * ForwardDesc. + * + */ +@Explain(displayName = "YSmart Forward") +public class YSmartForwardDesc implements Serializable { + private static final long serialVersionUID = 1L; + + private int realOutputInspectorIndicator; + + + + + public void setRealOutputInspectorIndicator(int realOutputInspectorIndicator){ + this.realOutputInspectorIndicator = realOutputInspectorIndicator; + } + + + + public int getRealOutputInspectorIndicator(){ + return this.realOutputInspectorIndicator; + } + + public YSmartForwardDesc() { + realOutputInspectorIndicator = 0; + } + + + public YSmartForwardDesc(int realOutputInspectorIndicator) { + this.realOutputInspectorIndicator = realOutputInspectorIndicator; + } + + + + + + + +} Index: ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java (revision 1132812) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java (working copy) @@ -70,6 +70,7 @@ private Long minSplitSizePerRack; private boolean needsTagging; + private boolean needsOpsTagging;//Yin private boolean hadoopSupportsSplittable; private MapredLocalWork mapLocalWork; @@ -333,6 +334,15 @@ this.needsTagging = needsTagging; } + @Explain(displayName = "Needs Ops Tagging", normalExplain = false) + public boolean getNeedsOpsTagging() { + return needsOpsTagging; + } + + public void setNeedsOpsTagging(boolean needsOpsTagging) { + this.needsOpsTagging = needsOpsTagging; + } + public boolean getHadoopSupportsSplittable() { return hadoopSupportsSplittable; } Index: ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartFakeReduceSinkDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartFakeReduceSinkDesc.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartFakeReduceSinkDesc.java (revision 0) @@ -0,0 +1,228 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.io.Serializable; +import java.util.List; + +/** + * ReduceSinkDesc. + * + */ +@Explain(displayName = "YSmart Fake Reduce Output Operator") +public class YSmartFakeReduceSinkDesc implements Serializable { + private static final long serialVersionUID = 1L; + /** + * Key columns are passed to reducer in the "key". + */ + private java.util.ArrayList keyCols; + private java.util.ArrayList outputKeyColumnNames; + private List> distinctColumnIndices; + /** + * Value columns are passed to reducer in the "value". + */ + private java.util.ArrayList valueCols; + private java.util.ArrayList outputValueColumnNames; + /** + * Describe how to serialize the key. + */ + private TableDesc keySerializeInfo; + /** + * Describe how to serialize the value. + */ + private TableDesc valueSerializeInfo; + + /** + * The tag for this reducesink descriptor. + */ + private int tag; + + /** + * Number of distribution keys. + */ + private int numDistributionKeys; + + /** + * The partition columns (CLUSTER BY or DISTRIBUTE BY in Hive language). + * Partition columns decide the reducer that the current row goes to. + * Partition columns are not passed to reducer. + */ + private java.util.ArrayList partitionCols; + + private int numReducers; + + public YSmartFakeReduceSinkDesc() { + } + + public YSmartFakeReduceSinkDesc(java.util.ArrayList keyCols, + int numDistributionKeys, + java.util.ArrayList valueCols, + java.util.ArrayList outputKeyColumnNames, + List> distinctColumnIndices, + java.util.ArrayList outputValueColumnNames, int tag, + java.util.ArrayList partitionCols, int numReducers, + final TableDesc keySerializeInfo, final TableDesc valueSerializeInfo) { + this.keyCols = keyCols; + this.numDistributionKeys = numDistributionKeys; + this.valueCols = valueCols; + this.outputKeyColumnNames = outputKeyColumnNames; + this.outputValueColumnNames = outputValueColumnNames; + this.tag = tag; + this.numReducers = numReducers; + this.partitionCols = partitionCols; + this.keySerializeInfo = keySerializeInfo; + this.valueSerializeInfo = valueSerializeInfo; + this.distinctColumnIndices = distinctColumnIndices; + } + + public YSmartFakeReduceSinkDesc(ReduceSinkDesc reduceSinkDesc){ + this.keyCols = reduceSinkDesc.getKeyCols(); + this.numDistributionKeys = reduceSinkDesc.getNumDistributionKeys(); + this.valueCols = reduceSinkDesc.getValueCols(); + this.outputKeyColumnNames = reduceSinkDesc.getOutputKeyColumnNames(); + this.outputValueColumnNames = reduceSinkDesc.getOutputValueColumnNames(); + this.tag = reduceSinkDesc.getTag(); + this.numReducers = reduceSinkDesc.getNumReducers(); + this.partitionCols = reduceSinkDesc.getPartitionCols(); + this.keySerializeInfo = reduceSinkDesc.getKeySerializeInfo(); + this.valueSerializeInfo = reduceSinkDesc.getValueSerializeInfo(); + this.distinctColumnIndices = reduceSinkDesc.getDistinctColumnIndices(); + } + + public java.util.ArrayList getOutputKeyColumnNames() { + return outputKeyColumnNames; + } + + public void setOutputKeyColumnNames( + java.util.ArrayList outputKeyColumnNames) { + this.outputKeyColumnNames = outputKeyColumnNames; + } + + public java.util.ArrayList getOutputValueColumnNames() { + return outputValueColumnNames; + } + + public void setOutputValueColumnNames( + java.util.ArrayList outputValueColumnNames) { + this.outputValueColumnNames = outputValueColumnNames; + } + + @Explain(displayName = "key expressions") + public java.util.ArrayList getKeyCols() { + return keyCols; + } + + public void setKeyCols(final java.util.ArrayList keyCols) { + this.keyCols = keyCols; + } + + public int getNumDistributionKeys() { + return this.numDistributionKeys; + } + + public void setNumDistributionKeys(int numKeys) { + this.numDistributionKeys = numKeys; + } + + @Explain(displayName = "value expressions") + public java.util.ArrayList getValueCols() { + return valueCols; + } + + public void setValueCols(final java.util.ArrayList valueCols) { + this.valueCols = valueCols; + } + + @Explain(displayName = "Map-reduce partition columns") + public java.util.ArrayList getPartitionCols() { + return partitionCols; + } + + public void setPartitionCols( + final java.util.ArrayList partitionCols) { + this.partitionCols = partitionCols; + } + + @Explain(displayName = "tag") + public int getTag() { + return tag; + } + + public void setTag(int tag) { + this.tag = tag; + } + + /** + * Returns the number of reducers for the map-reduce job. -1 means to decide + * the number of reducers at runtime. This enables Hive to estimate the number + * of reducers based on the map-reduce input data size, which is only + * available right before we start the map-reduce job. + */ + public int getNumReducers() { + return numReducers; + } + + public void setNumReducers(int numReducers) { + this.numReducers = numReducers; + } + + public TableDesc getKeySerializeInfo() { + return keySerializeInfo; + } + + public void setKeySerializeInfo(TableDesc keySerializeInfo) { + this.keySerializeInfo = keySerializeInfo; + } + + public TableDesc getValueSerializeInfo() { + return valueSerializeInfo; + } + + public void setValueSerializeInfo(TableDesc valueSerializeInfo) { + this.valueSerializeInfo = valueSerializeInfo; + } + + /** + * Returns the sort order of the key columns. + * + * @return null, which means ascending order for all key columns, or a String + * of the same length as key columns, that consists of only "+" + * (ascending order) and "-" (descending order). + */ + @Explain(displayName = "sort order") + public String getOrder() { + return keySerializeInfo.getProperties().getProperty( + org.apache.hadoop.hive.serde.Constants.SERIALIZATION_SORT_ORDER); + } + + public void setOrder(String orderStr) { + keySerializeInfo.getProperties().setProperty( + org.apache.hadoop.hive.serde.Constants.SERIALIZATION_SORT_ORDER, + orderStr); + } + + public List> getDistinctColumnIndices() { + return distinctColumnIndices; + } + + public void setDistinctColumnIndices( + List> distinctColumnIndices) { + this.distinctColumnIndices = distinctColumnIndices; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartReduceSinkDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartReduceSinkDesc.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartReduceSinkDesc.java (revision 0) @@ -0,0 +1,311 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** + * ReduceSinkDesc. + * + */ +@Explain(displayName = "YSmart Reduce Output Operator") +public class YSmartReduceSinkDesc implements Serializable { + private static final long serialVersionUID = 1L; + /** + * Key columns are passed to reducer in the "key". + */ + private java.util.ArrayList keyCols; + private java.util.ArrayList outputKeyColumnNames; + private List> distinctColumnIndices; + /** + * Value columns are passed to reducer in the "value". + */ + private java.util.ArrayList valueCols; + private java.util.ArrayList outputValueColumnNames; + /** + * Describe how to serialize the key. + */ + private TableDesc keySerializeInfo; + /** + * Describe how to serialize the value. + */ + private TableDesc valueSerializeInfo; + + /** + * The tag for this reducesink descriptor. + */ + private int tag; + + /** + * The array of tags which indicate which operations the output does not belong to. + */ + private ArrayList operationIndicators; + + /** + * Number of distribution keys. + */ + private int numDistributionKeys; + + /** + * The partition columns (CLUSTER BY or DISTRIBUTE BY in Hive language). + * Partition columns decide the reducer that the current row goes to. + * Partition columns are not passed to reducer. + */ + private java.util.ArrayList partitionCols; + + private int numReducers; + + private ArrayList staticOpTags; //Yin: Tags used for indicating which ops this row does not belong to + + public YSmartReduceSinkDesc() { + } + /* + public YSmartReduceSinkDesc(List reduceSinkDescs, java.util.ArrayList staticOpTags) { + this.keyCols = new java.util.ArrayList(); + this.numDistributionKeys = 0; + this.valueCols = new java.util.ArrayList(); + this.outputKeyColumnNames = new java.util.ArrayList(); + this.outputValueColumnNames = new java.util.ArrayList(); + this.tag = 0; + this.operationIndicators = new java.util.ArrayList(); + this.numReducers = 0; + this.partitionCols = new java.util.ArrayList(); + this.keySerializeInfo = null; + this.valueSerializeInfo = null; + this.distinctColumnIndices = null; + this.staticOpTags = staticOpTags; + + for (ReduceSinkDesc reduceSinkDesc : reduceSinkDescs){ + this.keyCols.removeAll(reduceSinkDesc.getKeyCols()); + this.keyCols.addAll(reduceSinkDesc.getKeyCols()); + this.numDistributionKeys = reduceSinkDesc.getNumDistributionKeys(); + this.valueCols.removeAll(reduceSinkDesc.getValueCols()); + this.valueCols.addAll(reduceSinkDesc.getValueCols()); + this.outputKeyColumnNames.removeAll(reduceSinkDesc.getOutputKeyColumnNames()); + this.outputKeyColumnNames.addAll(reduceSinkDesc.getOutputKeyColumnNames()); + this.outputValueColumnNames.removeAll(reduceSinkDesc.getOutputValueColumnNames()); + this.outputValueColumnNames.addAll(reduceSinkDesc.getOutputValueColumnNames()); + if (reduceSinkDesc.getTag() > this.tag){ + this.tag = reduceSinkDesc.getTag(); + } + if (reduceSinkDesc.getNumReducers() > this.numReducers){ + this.numReducers = reduceSinkDesc.getNumReducers(); + } + this.partitionCols.removeAll(reduceSinkDesc.getPartitionCols()); + this.partitionCols.addAll(reduceSinkDesc.getPartitionCols()); + this.keySerializeInfo = reduceSinkDesc.getKeySerializeInfo(); + this.valueSerializeInfo = reduceSinkDesc.getValueSerializeInfo(); + if (reduceSinkDesc.getDistinctColumnIndices() != null){ + this.distinctColumnIndices = new ArrayList>(); + this.distinctColumnIndices.removeAll(reduceSinkDesc.getDistinctColumnIndices()); + this.distinctColumnIndices.addAll(reduceSinkDesc.getDistinctColumnIndices()); + } + + } + + } + */ + + public YSmartReduceSinkDesc(java.util.ArrayList keyCols, + int numDistributionKeys, + java.util.ArrayList valueCols, + java.util.ArrayList outputKeyColumnNames, + List> distinctColumnIndices, + java.util.ArrayList outputValueColumnNames, int tag, + java.util.ArrayList partitionCols, int numReducers, + final TableDesc keySerializeInfo, final TableDesc valueSerializeInfo) { + this.keyCols = keyCols; + this.numDistributionKeys = numDistributionKeys; + this.valueCols = valueCols; + this.outputKeyColumnNames = outputKeyColumnNames; + this.outputValueColumnNames = outputValueColumnNames; + this.tag = tag; + this.numReducers = numReducers; + this.partitionCols = partitionCols; + this.keySerializeInfo = keySerializeInfo; + this.valueSerializeInfo = valueSerializeInfo; + this.distinctColumnIndices = distinctColumnIndices; + this.operationIndicators = null; + this.staticOpTags = null; + } + + public YSmartReduceSinkDesc(java.util.ArrayList keyCols, + int numDistributionKeys, + java.util.ArrayList valueCols, + java.util.ArrayList outputKeyColumnNames, + List> distinctColumnIndices, + java.util.ArrayList outputValueColumnNames, int tag, + java.util.ArrayList partitionCols, int numReducers, + final TableDesc keySerializeInfo, final TableDesc valueSerializeInfo, + java.util.ArrayList staticOpTags, + java.util.ArrayList operationIndicators) { + this.keyCols = keyCols; + this.numDistributionKeys = numDistributionKeys; + this.valueCols = valueCols; + this.outputKeyColumnNames = outputKeyColumnNames; + this.outputValueColumnNames = outputValueColumnNames; + this.tag = tag; + this.operationIndicators = operationIndicators; + this.numReducers = numReducers; + this.partitionCols = partitionCols; + this.keySerializeInfo = keySerializeInfo; + this.valueSerializeInfo = valueSerializeInfo; + this.distinctColumnIndices = distinctColumnIndices; + this.staticOpTags = staticOpTags; + } + + public java.util.ArrayList getStaticOpTags(){ + return staticOpTags; + } + + + public java.util.ArrayList getOutputKeyColumnNames() { + return outputKeyColumnNames; + } + + public void setOutputKeyColumnNames( + java.util.ArrayList outputKeyColumnNames) { + this.outputKeyColumnNames = outputKeyColumnNames; + } + + public java.util.ArrayList getOutputValueColumnNames() { + return outputValueColumnNames; + } + + public void setOutputValueColumnNames( + java.util.ArrayList outputValueColumnNames) { + this.outputValueColumnNames = outputValueColumnNames; + } + + @Explain(displayName = "key expressions") + public java.util.ArrayList getKeyCols() { + return keyCols; + } + + public void setKeyCols(final java.util.ArrayList keyCols) { + this.keyCols = keyCols; + } + + public int getNumDistributionKeys() { + return this.numDistributionKeys; + } + + public void setNumDistributionKeys(int numKeys) { + this.numDistributionKeys = numKeys; + } + + @Explain(displayName = "value expressions") + public java.util.ArrayList getValueCols() { + return valueCols; + } + + public void setValueCols(final java.util.ArrayList valueCols) { + this.valueCols = valueCols; + } + + @Explain(displayName = "Map-reduce partition columns") + public java.util.ArrayList getPartitionCols() { + return partitionCols; + } + + public void setPartitionCols( + final java.util.ArrayList partitionCols) { + this.partitionCols = partitionCols; + } + + @Explain(displayName = "tag") + public int getTag() { + return tag; + } + + public void setTag(int tag) { + this.tag = tag; + } + + + @Explain(displayName = "operationIndicators") + public ArrayList getOperationIndicators() { + return operationIndicators; + } + + public void setTag(ArrayList operationIndicators) { + this.operationIndicators = operationIndicators; + } + + + /** + * Returns the number of reducers for the map-reduce job. -1 means to decide + * the number of reducers at runtime. This enables Hive to estimate the number + * of reducers based on the map-reduce input data size, which is only + * available right before we start the map-reduce job. + */ + public int getNumReducers() { + return numReducers; + } + + public void setNumReducers(int numReducers) { + this.numReducers = numReducers; + } + + public TableDesc getKeySerializeInfo() { + return keySerializeInfo; + } + + public void setKeySerializeInfo(TableDesc keySerializeInfo) { + this.keySerializeInfo = keySerializeInfo; + } + + public TableDesc getValueSerializeInfo() { + return valueSerializeInfo; + } + + public void setValueSerializeInfo(TableDesc valueSerializeInfo) { + this.valueSerializeInfo = valueSerializeInfo; + } + + /** + * Returns the sort order of the key columns. + * + * @return null, which means ascending order for all key columns, or a String + * of the same length as key columns, that consists of only "+" + * (ascending order) and "-" (descending order). + */ + @Explain(displayName = "sort order") + public String getOrder() { + return keySerializeInfo.getProperties().getProperty( + org.apache.hadoop.hive.serde.Constants.SERIALIZATION_SORT_ORDER); + } + + public void setOrder(String orderStr) { + keySerializeInfo.getProperties().setProperty( + org.apache.hadoop.hive.serde.Constants.SERIALIZATION_SORT_ORDER, + orderStr); + } + + public List> getDistinctColumnIndices() { + return distinctColumnIndices; + } + + public void setDistinctColumnIndices( + List> distinctColumnIndices) { + this.distinctColumnIndices = distinctColumnIndices; + } +} Index: ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartDispatchDesc.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartDispatchDesc.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/plan/YSmartDispatchDesc.java (revision 0) @@ -0,0 +1,78 @@ +package org.apache.hadoop.hive.ql.plan; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map.Entry; + + +/** + * YSmart dispatch operator Descriptor implementation. + * + */ +@Explain(displayName = "YSmart Dispatch Operator") +public class YSmartDispatchDesc implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + + private HashMap> dispatchConf; + private HashMap> dispatchValueSelectDescConf; + private HashMap> dispatchKeySelectDescConf; + + public YSmartDispatchDesc(){ + this.dispatchConf = new HashMap>(); + this.dispatchValueSelectDescConf = new HashMap>(); + this.dispatchKeySelectDescConf = new HashMap>(); + + } + + public YSmartDispatchDesc(HashMap> dispatchConf){ + this.dispatchConf = dispatchConf; + this.dispatchValueSelectDescConf = new HashMap>(); + this.dispatchKeySelectDescConf = new HashMap>(); + for(Entry> entry: this.dispatchConf.entrySet()){ + HashMap tmp = new HashMap(); + for(Integer child: entry.getValue().keySet()){ + tmp.put(child, new SelectDesc(true)); + } + this.dispatchValueSelectDescConf.put(entry.getKey(), tmp); + this.dispatchKeySelectDescConf.put(entry.getKey(), tmp); + } + } + + public YSmartDispatchDesc(HashMap> dispatchConf, + HashMap> dispatchKeySelectDescConf, + HashMap> dispatchValueSelectDescConf){ + this.dispatchConf = dispatchConf; + this.dispatchValueSelectDescConf = dispatchValueSelectDescConf; + this.dispatchKeySelectDescConf = dispatchKeySelectDescConf; + } + + public void setDispatchConf(HashMap> dispatchConf){ + this.dispatchConf = dispatchConf; + } + + public HashMap> getDispatchConf(){ + return this.dispatchConf; + } + + public void setDispatchValueSelectDescConf(HashMap> dispatchValueSelectDescConf){ + this.dispatchValueSelectDescConf = dispatchValueSelectDescConf; + } + + public HashMap> getDispatchValueSelectDescConf(){ + return this.dispatchValueSelectDescConf; + } + + public void setDispatchKeySelectDescConf(HashMap> dispatchKeySelectDescConf){ + this.dispatchKeySelectDescConf = dispatchKeySelectDescConf; + } + + public HashMap> getDispatchKeySelectDescConf() { + return this.dispatchKeySelectDescConf; + } + +} Index: ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (revision 1132812) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (working copy) @@ -27,9 +27,9 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; import java.util.TreeSet; +import java.util.Map.Entry; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; @@ -68,6 +68,7 @@ import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.YSmartReduceSinkOperator; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; @@ -92,7 +93,6 @@ import org.apache.hadoop.hive.ql.optimizer.GenMRFileSink1; import org.apache.hadoop.hive.ql.optimizer.GenMROperator; import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext; -import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx; import org.apache.hadoop.hive.ql.optimizer.GenMRRedSink1; import org.apache.hadoop.hive.ql.optimizer.GenMRRedSink2; import org.apache.hadoop.hive.ql.optimizer.GenMRRedSink3; @@ -102,6 +102,9 @@ import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; import org.apache.hadoop.hive.ql.optimizer.MapJoinFactory; import org.apache.hadoop.hive.ql.optimizer.Optimizer; +import org.apache.hadoop.hive.ql.optimizer.YSmartGenMRRedSink1; +import org.apache.hadoop.hive.ql.optimizer.YSmartGenMRRedSink2; +import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx; import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalOptimizer; import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner; @@ -122,7 +125,6 @@ import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.FilterDesc; -import org.apache.hadoop.hive.ql.plan.FilterDesc.sampleDesc; import org.apache.hadoop.hive.ql.plan.ForwardDesc; import org.apache.hadoop.hive.ql.plan.GroupByDesc; import org.apache.hadoop.hive.ql.plan.HiveOperation; @@ -145,12 +147,13 @@ import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.hive.ql.plan.UDTFDesc; import org.apache.hadoop.hive.ql.plan.UnionDesc; +import org.apache.hadoop.hive.ql.plan.FilterDesc.sampleDesc; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.ResourceType; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFHash; import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode; import org.apache.hadoop.hive.serde.Constants; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe; @@ -158,9 +161,9 @@ import org.apache.hadoop.hive.serde2.SerDeUtils; import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; import org.apache.hadoop.hive.serde2.objectinspector.StructField; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; @@ -179,7 +182,7 @@ private List loadTableWork; private List loadFileWork; private Map joinContext; - private final HashMap topToTable; + private HashMap topToTable; private QB qb; private ASTNode ast; private int destTableId; @@ -285,6 +288,7 @@ opParseCtx.clear(); groupOpToInputTables.clear(); prunedPartitions.clear(); + topToTable.clear(); } public void init(ParseContext pctx) { @@ -292,6 +296,7 @@ opToPartList = pctx.getOpToPartList(); opToSamplePruner = pctx.getOpToSamplePruner(); topOps = pctx.getTopOps(); + topToTable = pctx.getTopToTable(); topSelOps = pctx.getTopSelOps(); opParseCtx = pctx.getOpParseCtx(); loadTableWork = pctx.getLoadTableWork(); @@ -2823,15 +2828,20 @@ for (int i = 1; i < value.getChildCount(); i++) { ASTNode parameter = (ASTNode) value.getChild(i); if (reduceSinkOutputRowResolver.getExpression(parameter) == null) { - reduceValues.add(genExprNodeDesc(parameter, - reduceSinkInputRowResolver)); + ExprNodeDesc expr = genExprNodeDesc(parameter, + reduceSinkInputRowResolver); + reduceValues.add(expr); outputValueColumnNames .add(getColumnInternalName(reduceValues.size() - 1)); String field = Utilities.ReduceField.VALUE.toString() + "." + getColumnInternalName(reduceValues.size() - 1); - reduceSinkOutputRowResolver.putExpression(parameter, new ColumnInfo(field, + ColumnInfo colInfo = new ColumnInfo(field, reduceValues.get(reduceValues.size() - 1).getTypeInfo(), null, - false)); + false); //Yin + reduceSinkOutputRowResolver.putExpression(parameter, colInfo); + + //System.out.println("in gby rs:" + expr.getExprString());//Yin + colExprMap.put(colInfo.getInternalName(),expr);//Yin } } } @@ -2843,14 +2853,19 @@ TypeInfo type = reduceSinkInputRowResolver.getColumnInfos().get( inputField).getType(); - reduceValues.add(new ExprNodeColumnDesc(type, - getColumnInternalName(inputField), "", false)); + ExprNodeDesc expr = new ExprNodeColumnDesc(type, + getColumnInternalName(inputField), "", false); + reduceValues.add(expr); inputField++; outputValueColumnNames.add(getColumnInternalName(reduceValues.size() - 1)); String field = Utilities.ReduceField.VALUE.toString() + "." + getColumnInternalName(reduceValues.size() - 1); + ColumnInfo colInfo = new ColumnInfo(field, + type, null, false);//Yin reduceSinkOutputRowResolver.putExpression(entry.getValue(), - new ColumnInfo(field, type, null, false)); + colInfo); + + colExprMap.put(colInfo.getInternalName(),expr);//Yin } } @@ -5788,7 +5803,8 @@ // the data to shuffle curr = insertSelectAllPlanForGroupBy(dest, curr); if (conf.getVar(HiveConf.ConfVars.HIVEMAPSIDEAGGREGATE) - .equalsIgnoreCase("true")) { + .equalsIgnoreCase("true") && conf.getVar(HiveConf.ConfVars.YSMARTOPTIMIZER) + .equalsIgnoreCase("false")) { if (conf.getVar(HiveConf.ConfVars.HIVEGROUPBYSKEW) .equalsIgnoreCase("false")) { curr = genGroupByPlanMapAggr1MR(dest, qb, curr); @@ -6858,7 +6874,12 @@ MapJoinFactory.getMapJoinMapJoin()); opRules.put(new RuleRegExp(new String("R11"), "MAPJOIN%SEL%"), MapJoinFactory.getMapJoin()); + opRules.put(new RuleRegExp(new String("R12"), "TS%.*YRS%"), + new YSmartGenMRRedSink1()); //Yin: Rule for translate YSmartCompositeOperator + opRules.put(new RuleRegExp(new String("R13"), "YRS%.*RS%"), + new YSmartGenMRRedSink2()); + // The dispatcher fires the processor corresponding to the closest matching // rule and passes the context along Dispatcher disp = new DefaultRuleDispatcher(new GenMROperator(), opRules, @@ -7042,6 +7063,10 @@ topOp.setChildOperators(null); } + if (topOp instanceof YSmartReduceSinkOperator) { + topOp.setChildOperators(null); + } + if (topOp.getChildOperators() == null) { return; } Index: ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java =================================================================== --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java (revision 1132812) +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java (working copy) @@ -6,8 +6,6 @@ package org.apache.hadoop.hive.ql.plan.api; -import java.util.Map; -import java.util.HashMap; import org.apache.thrift.TEnum; public enum OperatorType implements TEnum { @@ -28,7 +26,13 @@ LATERALVIEWJOIN(14), LATERALVIEWFORWARD(15), HASHTABLESINK(16), - HASHTABLEDUMMY(17); + HASHTABLEDUMMY(17), + YSMARTCOMPOSITE(18), + YSMARTREDUCESINK(19), + YSmartManualForward(20), + YSmartDispatch(21), + YSMARTFAKEREDUCESINK(22), + YSMARTFORWARD(23); private final int value; @@ -47,7 +51,7 @@ * Find a the enum type by its integer value, as defined in the Thrift IDL. * @return null if the value is not found. */ - public static OperatorType findByValue(int value) { + public static OperatorType findByValue(int value) { switch (value) { case 0: return JOIN; @@ -85,6 +89,18 @@ return HASHTABLESINK; case 17: return HASHTABLEDUMMY; + case 18: + return YSMARTCOMPOSITE; + case 19: + return YSMARTREDUCESINK; + case 20: + return YSmartManualForward; + case 21: + return YSmartDispatch; + case 22: + return YSMARTFAKEREDUCESINK; + case 23: + return YSMARTFORWARD; default: return null; }